diff --git a/README.md b/README.md index 353d18576..a39300764 100644 --- a/README.md +++ b/README.md @@ -11,6 +11,29 @@ To build a development version you'll need a recent version of Kafka as well as You can build kafka-connect-elasticsearch with Maven using the standard lifecycle phases. +# Configuring +## Creating an Elasticsearch user and assigning required privileges +### Create an Elasticsearch role +``` +curl -u elastic:elastic -X POST "localhost:9200/_security/role/es_sink_connector_role?pretty" -H 'Content-Type: application/json' -d' +{ + "indices": [ + { + "names": [ "*" ], + "privileges": ["create_index", "read", "write", "view_index_metadata"] + } + ] +}' +``` +### Create an Elasticsearch user +``` +curl -u elastic:elastic -X POST "localhost:9200/_security/user/es_sink_connector_user?pretty" -H 'Content-Type: application/json' -d' +{ + "password" : "seCret-secUre-PaSsW0rD", + "roles" : [ "es_sink_connector_role" ] +}' +``` + # Contribute - Source Code: https://github.com/confluentinc/kafka-connect-elasticsearch diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml index 25f658354..7e6bcb489 100644 --- a/checkstyle/suppressions.xml +++ b/checkstyle/suppressions.xml @@ -8,30 +8,18 @@ - - - - - - diff --git a/pom.xml b/pom.xml index 716452ff9..a31bd293f 100644 --- a/pom.xml +++ b/pom.xml @@ -5,12 +5,12 @@ io.confluent common - 6.0.0 + 6.0.3 io.confluent kafka-connect-elasticsearch - 10.0.5-SNAPSHOT + 11.0.15-SNAPSHOT jar kafka-connect-elasticsearch @@ -34,24 +34,25 @@ scm:git:git://github.com/confluentinc/kafka-connect-elasticsearch.git scm:git:git@github.com:confluentinc/kafka-connect-elasticsearch.git https://github.com/confluentinc/kafka-connect-elasticsearch - 10.0.x + 11.0.x - 2.4.6 - 5.5.4 + 7.9.3 1.3 2.28.2 - 6.3.1 + 2.8.6 1.15.3 0.11.1 2.5.3 + 3.3.0 + 2.0.0-M2 + 2.10.5.1 6.1.6 http://packages.confluent.io/maven/ 1.15 - 1.28 2.16.0 true @@ -76,9 +77,48 @@ provided - io.searchbox - jest - ${jest.version} + org.elasticsearch.client + elasticsearch-rest-high-level-client + ${es.version} + + + org.elasticsearch + elasticsearch + ${es.version} + + + org.apache.logging.log4j + log4j-api + 2.16.0 + + + + com.fasterxml.jackson.dataformat + jackson-dataformat-cbor + + + + commons-codec + commons-codec + 1.15 + + + + org.yaml + snakeyaml + 1.27 + + + + org.apache.httpcomponents + httpclient + 4.5.13 + + + com.google.code.gson + gson + ${gson.version} + test org.apache.httpcomponents @@ -104,7 +144,7 @@ org.mockito - mockito-core + mockito-inline ${mockito.version} test @@ -120,36 +160,6 @@ test - - org.apache.lucene - lucene-test-framework - ${lucene.version} - test - - - org.elasticsearch - elasticsearch - ${es.version} - test - test-jar - - - org.elasticsearch - elasticsearch - ${es.version} - test - - - org.apache.lucene - lucene-expressions - ${lucene.version} - test - - - com.google.guava - guava - ${guava.version} - org.apache.kafka connect-runtime @@ -199,9 +209,27 @@ test - org.yaml - snakeyaml - ${snakeyaml.version} + org.apache.hadoop + hadoop-minikdc + ${hadoop.version} + test + + + org.assertj + assertj-core + 3.20.2 + test + + + org.awaitility + awaitility + 4.1.0 + test + + + com.github.tomakehurst + wiremock-jre8 + 2.30.1 test @@ -253,6 +281,16 @@ + + + + com.fasterxml.jackson.core + jackson-databind + ${jackson.databind.version} + + + + @@ -376,7 +414,7 @@ logos/elasticsearch.jpg Confluent, Inc. - Confluent supports the Elasticsearch sink connector alongside community members as part of its Confluent Platform offering. + supported by Confluent as part of a Confluent Platform subscription.]]> https://docs.confluent.io/home/overview.html logos/confluent.png @@ -399,7 +437,7 @@ - Elasticsearch 2.x, 5.x, 6.x, or 7.x + Elasticsearch 7.x @@ -516,7 +554,7 @@ enabled and disabled cases with the same cert. So, disable this IT in jenkins. --> - **/ElasticsearchConnectorSecureIT.java + **/ElasticsearchConnectorSslIT.java @@ -640,4 +678,4 @@ - \ No newline at end of file + diff --git a/src/main/java/io/confluent/connect/elasticsearch/BulkIndexingClient.java b/src/main/java/io/confluent/connect/elasticsearch/BulkIndexingClient.java deleted file mode 100644 index ac7adde10..000000000 --- a/src/main/java/io/confluent/connect/elasticsearch/BulkIndexingClient.java +++ /dev/null @@ -1,43 +0,0 @@ -/* - * 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.connect.elasticsearch; - -import io.confluent.connect.elasticsearch.bulk.BulkClient; -import io.confluent.connect.elasticsearch.bulk.BulkRequest; -import io.confluent.connect.elasticsearch.bulk.BulkResponse; - -import java.io.IOException; -import java.util.List; - -public class BulkIndexingClient implements BulkClient { - - private final ElasticsearchClient client; - - public BulkIndexingClient(ElasticsearchClient client) { - this.client = client; - } - - @Override - public BulkRequest bulkRequest(List batch) { - return client.createBulkRequest(batch); - } - - @Override - public BulkResponse execute(BulkRequest bulk) throws IOException { - return client.executeBulk(bulk); - } - -} diff --git a/src/main/java/io/confluent/connect/elasticsearch/ConfigCallbackHandler.java b/src/main/java/io/confluent/connect/elasticsearch/ConfigCallbackHandler.java new file mode 100644 index 000000000..1c2676604 --- /dev/null +++ b/src/main/java/io/confluent/connect/elasticsearch/ConfigCallbackHandler.java @@ -0,0 +1,368 @@ +/* + * Copyright 2020 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.connect.elasticsearch; + +import com.sun.security.auth.module.Krb5LoginModule; +import java.security.AccessController; +import java.security.PrivilegedActionException; +import java.security.PrivilegedExceptionAction; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import javax.net.ssl.HostnameVerifier; +import javax.net.ssl.SSLContext; +import javax.security.auth.Subject; +import javax.security.auth.kerberos.KerberosPrincipal; +import javax.security.auth.login.AppConfigurationEntry; +import javax.security.auth.login.Configuration; +import javax.security.auth.login.LoginContext; +import org.apache.http.HttpHost; +import org.apache.http.auth.AuthSchemeProvider; +import org.apache.http.auth.AuthScope; +import org.apache.http.auth.KerberosCredentials; +import org.apache.http.auth.UsernamePasswordCredentials; +import org.apache.http.client.CredentialsProvider; +import org.apache.http.client.config.AuthSchemes; +import org.apache.http.client.config.RequestConfig; +import org.apache.http.config.Lookup; +import org.apache.http.config.Registry; +import org.apache.http.config.RegistryBuilder; +import org.apache.http.conn.ssl.NoopHostnameVerifier; +import org.apache.http.conn.ssl.SSLConnectionSocketFactory; +import org.apache.http.impl.auth.SPNegoSchemeFactory; +import org.apache.http.impl.client.BasicCredentialsProvider; +import org.apache.http.impl.nio.client.HttpAsyncClientBuilder; +import org.apache.http.impl.nio.conn.PoolingNHttpClientConnectionManager; +import org.apache.http.impl.nio.reactor.DefaultConnectingIOReactor; +import org.apache.http.impl.nio.reactor.IOReactorConfig; +import org.apache.http.nio.conn.NoopIOSessionStrategy; +import org.apache.http.nio.conn.SchemeIOSessionStrategy; +import org.apache.http.nio.conn.ssl.SSLIOSessionStrategy; +import org.apache.http.nio.reactor.ConnectingIOReactor; +import org.apache.http.nio.reactor.IOReactorException; +import org.apache.kafka.common.network.Mode; +import org.apache.kafka.common.security.ssl.SslFactory; +import org.apache.kafka.connect.errors.ConnectException; +import org.elasticsearch.client.RestClientBuilder.HttpClientConfigCallback; +import org.ietf.jgss.GSSCredential; +import org.ietf.jgss.GSSException; +import org.ietf.jgss.GSSManager; +import org.ietf.jgss.Oid; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class ConfigCallbackHandler implements HttpClientConfigCallback { + + private static final Logger log = LoggerFactory.getLogger(ConfigCallbackHandler.class); + + private static final Oid SPNEGO_OID = spnegoOid(); + + private final ElasticsearchSinkConnectorConfig config; + + public ConfigCallbackHandler(ElasticsearchSinkConnectorConfig config) { + this.config = config; + } + + /** + * Customizes the client according to the configurations and starts the connection reaping thread. + * + * @param builder the HttpAsyncClientBuilder + * @return the builder + */ + @Override + public HttpAsyncClientBuilder customizeHttpClient(HttpAsyncClientBuilder builder) { + RequestConfig requestConfig = RequestConfig.custom() + .setContentCompressionEnabled(config.compression()) + .setConnectTimeout(config.connectionTimeoutMs()) + .setConnectionRequestTimeout(config.readTimeoutMs()) + .setSocketTimeout(config.readTimeoutMs()) + .build(); + + builder.setConnectionManager(createConnectionManager()) + .setDefaultRequestConfig(requestConfig); + + configureAuthentication(builder); + + if (config.isKerberosEnabled()) { + configureKerberos(builder); + } + + if (config.isSslEnabled()) { + configureSslContext(builder); + } + + if (config.isKerberosEnabled() && config.isSslEnabled()) { + log.info("Using Kerberos and SSL connection to {}.", config.connectionUrls()); + } else if (config.isKerberosEnabled()) { + log.info("Using Kerberos connection to {}.", config.connectionUrls()); + } else if (config.isSslEnabled()) { + log.info("Using SSL connection to {}.", config.connectionUrls()); + } else { + log.info("Using unsecured connection to {}.", config.connectionUrls()); + } + + return builder; + } + + /** + * Configures HTTP authentication and proxy authentication according to the client configuration. + * + * @param builder the HttpAsyncClientBuilder + */ + private void configureAuthentication(HttpAsyncClientBuilder builder) { + CredentialsProvider credentialsProvider = new BasicCredentialsProvider(); + if (config.isAuthenticatedConnection()) { + config.connectionUrls().forEach(url -> credentialsProvider.setCredentials( + new AuthScope(HttpHost.create(url)), + new UsernamePasswordCredentials(config.username(), config.password().value()) + ) + ); + builder.setDefaultCredentialsProvider(credentialsProvider); + } + + if (config.isBasicProxyConfigured()) { + HttpHost proxy = new HttpHost(config.proxyHost(), config.proxyPort()); + builder.setProxy(proxy); + + if (config.isProxyWithAuthenticationConfigured()) { + credentialsProvider.setCredentials( + new AuthScope(proxy), + new UsernamePasswordCredentials(config.proxyUsername(), config.proxyPassword().value()) + ); + } + + builder.setDefaultCredentialsProvider(credentialsProvider); + } + } + + /** + * Creates a connection manager for the client. + * + * @return the connection manager + */ + private PoolingNHttpClientConnectionManager createConnectionManager() { + try { + PoolingNHttpClientConnectionManager cm; + IOReactorConfig ioReactorConfig = IOReactorConfig.custom() + .setConnectTimeout(config.connectionTimeoutMs()) + .setSoTimeout(config.readTimeoutMs()) + .build(); + ConnectingIOReactor ioReactor = new DefaultConnectingIOReactor(ioReactorConfig); + + if (config.isSslEnabled()) { + HostnameVerifier hostnameVerifier = config.shouldDisableHostnameVerification() + ? new NoopHostnameVerifier() + : SSLConnectionSocketFactory.getDefaultHostnameVerifier(); + Registry reg = RegistryBuilder.create() + .register("http", NoopIOSessionStrategy.INSTANCE) + .register("https", new SSLIOSessionStrategy(sslContext(), hostnameVerifier)) + .build(); + + cm = new PoolingNHttpClientConnectionManager(ioReactor, reg); + } else { + cm = new PoolingNHttpClientConnectionManager(ioReactor); + } + + // Allowing up to two http connections per processing thread to a given host + int maxPerRoute = Math.max(10, config.maxInFlightRequests() * 2); + cm.setDefaultMaxPerRoute(maxPerRoute); + // And for the global limit, with multiply the per-host limit + // by the number of potential different ES hosts + cm.setMaxTotal(maxPerRoute * config.connectionUrls().size()); + + log.debug("Connection pool config: maxPerRoute: {}, maxTotal {}", + cm.getDefaultMaxPerRoute(), + cm.getMaxTotal()); + + return cm; + } catch (IOReactorException e) { + throw new ConnectException("Unable to open ElasticsearchClient.", e); + } + } + + /** + * Configures the client to use Kerberos authentication. Overrides any proxy or basic auth + * credentials. + * + * @param builder the HttpAsyncClientBuilder to configure + * @return the configured builder + */ + private HttpAsyncClientBuilder configureKerberos(HttpAsyncClientBuilder builder) { + GSSManager gssManager = GSSManager.getInstance(); + Lookup authSchemeRegistry = + RegistryBuilder.create() + .register(AuthSchemes.SPNEGO, new SPNegoSchemeFactory()) + .build(); + builder.setDefaultAuthSchemeRegistry(authSchemeRegistry); + + try { + LoginContext loginContext = loginContext(); + GSSCredential credential = Subject.doAs( + loginContext.getSubject(), + (PrivilegedExceptionAction) () -> gssManager.createCredential( + null, + GSSCredential.DEFAULT_LIFETIME, + SPNEGO_OID, + GSSCredential.INITIATE_ONLY + ) + ); + CredentialsProvider credentialsProvider = new BasicCredentialsProvider(); + credentialsProvider.setCredentials( + new AuthScope( + AuthScope.ANY_HOST, + AuthScope.ANY_PORT, + AuthScope.ANY_REALM, + AuthSchemes.SPNEGO + ), + new KerberosCredentials(credential) + ); + builder.setDefaultCredentialsProvider(credentialsProvider); + } catch (PrivilegedActionException e) { + throw new ConnectException(e); + } + + return builder; + } + + /** + * Configures the client to use SSL if configured. + * + * @param builder the HttpAsyncClientBuilder + */ + private void configureSslContext(HttpAsyncClientBuilder builder) { + HostnameVerifier hostnameVerifier = config.shouldDisableHostnameVerification() + ? new NoopHostnameVerifier() + : SSLConnectionSocketFactory.getDefaultHostnameVerifier(); + + SSLContext sslContext = sslContext(); + builder.setSSLContext(sslContext); + builder.setSSLHostnameVerifier(hostnameVerifier); + builder.setSSLStrategy(new SSLIOSessionStrategy(sslContext, hostnameVerifier)); + } + + /** + * Gets the SslContext for the client. + */ + private SSLContext sslContext() { + SslFactory sslFactory = new SslFactory(Mode.CLIENT, null, false); + sslFactory.configure(config.sslConfigs()); + + try { + // try AK <= 2.2 first + log.debug("Trying AK 2.2 SslFactory methods."); + return (SSLContext) SslFactory.class.getDeclaredMethod("sslContext").invoke(sslFactory); + } catch (Exception e) { + // must be running AK 2.3+ + log.debug("Could not find AK 2.2 SslFactory methods. Trying AK 2.3+ methods for SslFactory."); + + Object sslEngine; + try { + // try AK <= 2.6 second + sslEngine = SslFactory.class.getDeclaredMethod("sslEngineBuilder").invoke(sslFactory); + log.debug("Using AK 2.2-2.5 SslFactory methods."); + } catch (Exception ex) { + // must be running AK 2.6+ + log.debug( + "Could not find AK 2.3-2.5 SslFactory methods. Trying AK 2.6+ methods for SslFactory." + ); + try { + sslEngine = SslFactory.class.getDeclaredMethod("sslEngineFactory").invoke(sslFactory); + log.debug("Using AK 2.6+ SslFactory methods."); + } catch (Exception exc) { + throw new ConnectException("Failed to find methods for SslFactory.", exc); + } + } + + try { + return (SSLContext) sslEngine + .getClass() + .getDeclaredMethod("sslContext") + .invoke(sslEngine); + } catch (Exception ex) { + throw new ConnectException("Could not create SSLContext.", ex); + } + } + } + + /** + * Logs in and returns a login context for the given kerberos user principle. + * + * @return the login context + * @throws PrivilegedActionException if the login failed + */ + private LoginContext loginContext() throws PrivilegedActionException { + Configuration conf = new Configuration() { + @Override + public AppConfigurationEntry[] getAppConfigurationEntry(String name) { + return new AppConfigurationEntry[] { + new AppConfigurationEntry( + Krb5LoginModule.class.getName(), + AppConfigurationEntry.LoginModuleControlFlag.REQUIRED, + kerberosConfigs() + ) + }; + } + }; + + return AccessController.doPrivileged( + (PrivilegedExceptionAction) () -> { + Subject subject = new Subject( + false, + Collections.singleton(new KerberosPrincipal(config.kerberosUserPrincipal())), + new HashSet<>(), + new HashSet<>() + ); + LoginContext loginContext = new LoginContext( + "ElasticsearchSinkConnector", + subject, + null, + conf + ); + loginContext.login(); + return loginContext; + } + ); + } + + /** + * Creates the Kerberos configurations. + * + * @return map of kerberos configs + */ + private Map kerberosConfigs() { + Map configs = new HashMap<>(); + configs.put("useTicketCache", "true"); + configs.put("renewTGT", "true"); + configs.put("useKeyTab", "true"); + configs.put("keyTab", config.keytabPath()); + //Krb5 in GSS API needs to be refreshed so it does not throw the error + //Specified version of key is not available + configs.put("refreshKrb5Config", "true"); + configs.put("principal", config.kerberosUserPrincipal()); + configs.put("storeKey", "false"); + configs.put("doNotPrompt", "true"); + return configs; + } + + private static Oid spnegoOid() { + try { + return new Oid("1.3.6.1.5.5.2"); + } catch (GSSException gsse) { + throw new ConnectException(gsse); + } + } +} diff --git a/src/main/java/io/confluent/connect/elasticsearch/DataConverter.java b/src/main/java/io/confluent/connect/elasticsearch/DataConverter.java index 47d731df5..40f77f003 100644 --- a/src/main/java/io/confluent/connect/elasticsearch/DataConverter.java +++ b/src/main/java/io/confluent/connect/elasticsearch/DataConverter.java @@ -15,7 +15,7 @@ package io.confluent.connect.elasticsearch; -import org.apache.kafka.common.config.ConfigDef; +import io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.BehaviorOnNullValues; import org.apache.kafka.connect.data.ConnectSchema; import org.apache.kafka.connect.data.Date; import org.apache.kafka.connect.data.Decimal; @@ -27,9 +27,18 @@ import org.apache.kafka.connect.data.Timestamp; import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.errors.DataException; +import org.apache.kafka.connect.header.Header; import org.apache.kafka.connect.json.JsonConverter; import org.apache.kafka.connect.sink.SinkRecord; import org.apache.kafka.connect.storage.Converter; +import org.apache.kafka.connect.storage.HeaderConverter; +import org.apache.kafka.connect.storage.SimpleHeaderConverter; +import org.elasticsearch.action.DocWriteRequest; +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; +import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.index.VersionType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -40,25 +49,23 @@ import java.util.Collections; import java.util.HashMap; import java.util.List; -import java.util.Locale; import java.util.Map; -import java.util.Objects; - -import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConstants.MAP_KEY; -import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConstants.MAP_VALUE; public class DataConverter { private static final Logger log = LoggerFactory.getLogger(DataConverter.class); + private static final Converter JSON_CONVERTER; + private static final HeaderConverter HEADER_CONVERTER = new SimpleHeaderConverter(); + protected static final String MAP_KEY = "key"; + protected static final String MAP_VALUE = "value"; static { JSON_CONVERTER = new JsonConverter(); JSON_CONVERTER.configure(Collections.singletonMap("schemas.enable", "false"), false); } - private final boolean useCompactMapEntries; - private final BehaviorOnNullValues behaviorOnNullValues; + private final ElasticsearchSinkConnectorConfig config; /** * Create a DataConverter, specifying how map entries with string keys within record @@ -67,19 +74,15 @@ public class DataConverter { * document such as {"key": "entryKey", "value": "entryValue"}. All map entries * with non-string keys are always written as nested documents. * - * @param useCompactMapEntries true for compact map entries with string keys, or false for - * the nested document form. - * @param behaviorOnNullValues behavior for handling records with null values; may not be null + * @param config connector config */ - public DataConverter(boolean useCompactMapEntries, BehaviorOnNullValues behaviorOnNullValues) { - this.useCompactMapEntries = useCompactMapEntries; - this.behaviorOnNullValues = - Objects.requireNonNull(behaviorOnNullValues, "behaviorOnNullValues cannot be null."); + public DataConverter(ElasticsearchSinkConnectorConfig config) { + this.config = config; } private String convertKey(Schema keySchema, Object key) { if (key == null) { - throw new ConnectException("Key is used as document id and can not be null."); + throw new DataException("Key is used as document id and can not be null."); } final Schema.Type schemaType; @@ -87,9 +90,7 @@ private String convertKey(Schema keySchema, Object key) { schemaType = ConnectSchema.schemaType(key.getClass()); if (schemaType == null) { throw new DataException( - "Java class " - + key.getClass() - + " does not have corresponding schema type." + "Java class " + key.getClass() + " does not have corresponding schema type." ); } } else { @@ -108,22 +109,11 @@ private String convertKey(Schema keySchema, Object key) { } } - public IndexableRecord convertRecord( - SinkRecord record, - String index, - String type, - boolean ignoreKey, - boolean ignoreSchema - ) { + public DocWriteRequest convertRecord(SinkRecord record, String index) { if (record.value() == null) { - switch (behaviorOnNullValues) { + switch (config.behaviorOnNullValues()) { case IGNORE: - log.trace( - "Ignoring record with null value at topic '{}', partition {}, offset {}", - record.topic(), - record.kafkaPartition(), - record.kafkaOffset() - ); + log.trace("Ignoring {} with null value.", recordString(record)); return null; case DELETE: if (record.key() == null) { @@ -135,68 +125,66 @@ public IndexableRecord convertRecord( // unique per message, we can be confident that there wouldn't be any corresponding // index present in ES to delete anyways. log.trace( - "Ignoring record with null key at topic '{}', partition {}, offset {}, since " - + "the record key is used as the ID of the index", - record.topic(), - record.kafkaPartition(), - record.kafkaOffset() + "Ignoring {} with null key, since the record key is used as the ID of the index", + recordString(record) ); return null; } - // Will proceed as normal, ultimately creating an IndexableRecord with a null payload - log.trace( - "Deleting from Elasticsearch record at topic '{}', partition {}, offset {}", - record.topic(), - record.kafkaPartition(), - record.kafkaOffset() - ); + // Will proceed as normal, ultimately creating a DeleteRequest + log.trace("Deleting {} from Elasticsearch", recordString(record)); break; case FAIL: - throw new DataException(String.format( - "Sink record with key of %s and null value encountered for topic/partition/offset " - + "%s/%s/%s (to ignore future records like this change the configuration property " - + "'%s' from '%s' to '%s')", - record.key(), - record.topic(), - record.kafkaPartition(), - record.kafkaOffset(), - ElasticsearchSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, - BehaviorOnNullValues.FAIL, - BehaviorOnNullValues.IGNORE - )); default: - throw new RuntimeException(String.format( - "Unknown value for %s enum: %s", - BehaviorOnNullValues.class.getSimpleName(), - behaviorOnNullValues - )); + throw new DataException( + String.format( + "%s with key of %s and null value encountered (to ignore future records like" + + " this change the configuration property '%s' from '%s' to '%s')", + recordString(record), + record.key(), + ElasticsearchSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, + BehaviorOnNullValues.FAIL, + BehaviorOnNullValues.IGNORE + ) + ); } } - final String id; - if (ignoreKey) { - id = record.topic() - + "+" + String.valueOf((int) record.kafkaPartition()) - + "+" + String.valueOf(record.kafkaOffset()); - } else { - id = convertKey(record.keySchema(), record.key()); + final String payload = getPayload(record); + final String id = config.shouldIgnoreKey(record.topic()) + ? String.format("%s+%d+%d", record.topic(), record.kafkaPartition(), record.kafkaOffset()) + : convertKey(record.keySchema(), record.key()); + + // delete + if (record.value() == null) { + return maybeAddExternalVersioning(new DeleteRequest(index).id(id), record); } - final String payload = getPayload(record, ignoreSchema); - final Long version = ignoreKey ? null : record.kafkaOffset(); - return new IndexableRecord(new Key(index, type, id), payload, version); + // index + switch (config.writeMethod()) { + case UPSERT: + return new UpdateRequest(index, id) + .doc(payload, XContentType.JSON) + .upsert(payload, XContentType.JSON) + .retryOnConflict(Math.min(config.maxInFlightRequests(), 5)); + case INSERT: + return maybeAddExternalVersioning( + new IndexRequest(index).id(id).source(payload, XContentType.JSON), + record + ); + default: + return null; // shouldn't happen + } } - private String getPayload(SinkRecord record, boolean ignoreSchema) { + private String getPayload(SinkRecord record) { if (record.value() == null) { return null; } - Schema schema = ignoreSchema + Schema schema = config.shouldIgnoreSchema(record.topic()) ? record.valueSchema() : preProcessSchema(record.valueSchema()); - - Object value = ignoreSchema + Object value = config.shouldIgnoreSchema(record.topic()) ? record.value() : preProcessValue(record.value(), record.valueSchema(), schema); @@ -204,6 +192,44 @@ private String getPayload(SinkRecord record, boolean ignoreSchema) { return new String(rawJsonPayload, StandardCharsets.UTF_8); } + /** + * In many cases, we explicitly set the record version using the topic's offset. + * This version will, in turn, be checked by Elasticsearch and will throw a versioning + * error if the request represents an equivalent or older version of the record. + * + * @param request the request currently being constructed for `record` + * @param record the record to be processed + * @return the (possibly modified) request which was passed in + */ + private DocWriteRequest maybeAddExternalVersioning( + DocWriteRequest request, + SinkRecord record + ) { + if (!config.shouldIgnoreKey(record.topic())) { + request.versionType(VersionType.EXTERNAL); + if (config.hasExternalVersionHeader()) { + final Header versionHeader = record.headers().lastWithName(config.externalVersionHeader()); + final byte[] versionValue = HEADER_CONVERTER.fromConnectHeader( + record.topic(), + versionHeader.key(), + versionHeader.schema(), + versionHeader.value() + ); + try { + //fromConnectHeader byte output is UTF_8 + request.version(Long.parseLong(new String(versionValue, StandardCharsets.UTF_8))); + } catch (NumberFormatException e) { + throw new ConnectException("Error converting to long: " + + new String(versionValue, StandardCharsets.UTF_8), e); + } + } else { + request.version(record.kafkaOffset()); + } + } + + return request; + } + // We need to pre process the Kafka Connect schema before converting to JSON as Elasticsearch // expects a different JSON format from the current JSON converter provides. Rather than // completely rewrite a converter for Elasticsearch, we will refactor the JSON converter to @@ -255,7 +281,7 @@ private Schema preProcessMapSchema(Schema schema) { String valueName = valueSchema.name() == null ? valueSchema.type().name() : valueSchema.name(); Schema preprocessedKeySchema = preProcessSchema(keySchema); Schema preprocessedValueSchema = preProcessSchema(valueSchema); - if (useCompactMapEntries && keySchema.type() == Schema.Type.STRING) { + if (config.useCompactMapEntries() && keySchema.type() == Schema.Type.STRING) { SchemaBuilder result = SchemaBuilder.map(preprocessedKeySchema, preprocessedValueSchema); return copySchemaBasics(schema, result).build(); } @@ -357,7 +383,7 @@ private Object preProcessMapValue(Object value, Schema schema, Schema newSchema) Schema valueSchema = schema.valueSchema(); Schema newValueSchema = newSchema.valueSchema(); Map map = (Map) value; - if (useCompactMapEntries && keySchema.type() == Schema.Type.STRING) { + if (config.useCompactMapEntries() && keySchema.type() == Schema.Type.STRING) { Map processedMap = new HashMap<>(); for (Map.Entry entry: map.entrySet()) { processedMap.put( @@ -390,47 +416,12 @@ private Object preProcessStructValue(Object value, Schema schema, Schema newSche return newStruct; } - public enum BehaviorOnNullValues { - IGNORE, - DELETE, - FAIL; - - public static final BehaviorOnNullValues DEFAULT = IGNORE; - - // Want values for "behavior.on.null.values" property to be case-insensitive - public static final ConfigDef.Validator VALIDATOR = new ConfigDef.Validator() { - private final ConfigDef.ValidString validator = ConfigDef.ValidString.in(names()); - - @Override - public void ensureValid(String name, Object value) { - if (value instanceof String) { - value = ((String) value).toLowerCase(Locale.ROOT); - } - validator.ensureValid(name, value); - } - - // Overridden here so that ConfigDef.toEnrichedRst shows possible values correctly - @Override - public String toString() { - return validator.toString(); - } - - }; - - public static String[] names() { - BehaviorOnNullValues[] behaviors = values(); - String[] result = new String[behaviors.length]; - - for (int i = 0; i < behaviors.length; i++) { - result[i] = behaviors[i].toString(); - } - - return result; - } - - @Override - public String toString() { - return name().toLowerCase(Locale.ROOT); - } + private static String recordString(SinkRecord record) { + return String.format( + "record from topic=%s partition=%s offset=%s", + record.topic(), + record.kafkaPartition(), + record.kafkaOffset() + ); } } diff --git a/src/main/java/io/confluent/connect/elasticsearch/ElasticsearchClient.java b/src/main/java/io/confluent/connect/elasticsearch/ElasticsearchClient.java index 697f7a3fb..069f4058e 100644 --- a/src/main/java/io/confluent/connect/elasticsearch/ElasticsearchClient.java +++ b/src/main/java/io/confluent/connect/elasticsearch/ElasticsearchClient.java @@ -15,102 +15,597 @@ package io.confluent.connect.elasticsearch; -import com.google.gson.JsonObject; -import io.confluent.connect.elasticsearch.bulk.BulkRequest; -import io.confluent.connect.elasticsearch.bulk.BulkResponse; +import org.apache.http.HttpHost; +import org.apache.kafka.common.utils.Time; import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.errors.ConnectException; +import org.apache.kafka.connect.sink.ErrantRecordReporter; +import org.apache.kafka.connect.sink.SinkRecord; +import org.elasticsearch.ElasticsearchStatusException; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.DocWriteRequest; +import org.elasticsearch.action.bulk.BackoffPolicy; +import org.elasticsearch.action.bulk.BulkItemResponse; +import org.elasticsearch.action.bulk.BulkProcessor; +import org.elasticsearch.action.bulk.BulkProcessor.Listener; +import org.elasticsearch.action.bulk.BulkRequest; +import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.client.RequestOptions; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.RestHighLevelClient; +import org.elasticsearch.client.indices.CreateIndexRequest; +import org.elasticsearch.client.indices.GetIndexRequest; +import org.elasticsearch.client.indices.GetMappingsRequest; +import org.elasticsearch.client.indices.GetMappingsResponse; +import org.elasticsearch.client.indices.PutMappingRequest; +import org.elasticsearch.cluster.metadata.MappingMetadata; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.index.VersionType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; import java.util.List; import java.util.Set; +import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.BiConsumer; +import java.util.stream.Collectors; -public interface ElasticsearchClient extends AutoCloseable { +import io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.BehaviorOnMalformedDoc; - enum Version { - ES_V1, ES_V2, ES_V5, ES_V6, ES_V7 +import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.FLUSH_TIMEOUT_MS_CONFIG; +import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.MAX_BUFFERED_RECORDS_CONFIG; + +/** + * Based on Elasticsearch's BulkProcessor, which is responsible for building batches based on size + * and linger time (not grouped by partitions) and limiting the concurrency (max number of + * in-flight requests). + *
+ * Batch processing is asynchronous. BulkProcessor delegates the bulk calls to a separate thread + * pool. Retries are handled synchronously in each batch thread. + *
+ * If all the retries fail, the exception is reported via an atomic reference to an error, + * which is checked and thrown from a subsequent call to the task's put method and that results + * in failure of the task. + */ +@SuppressWarnings("checkstyle:ClassDataAbstractionCoupling") +public class ElasticsearchClient { + + private static final Logger log = LoggerFactory.getLogger(ElasticsearchClient.class); + + private static final long WAIT_TIME_MS = 10; + private static final long CLOSE_WAIT_TIME_MS = 5_000; + private static final String RESOURCE_ALREADY_EXISTS_EXCEPTION = + "resource_already_exists_exception"; + private static final String VERSION_CONFLICT_EXCEPTION = "version_conflict_engine_exception"; + + + + private static final Set MALFORMED_DOC_ERRORS = new HashSet<>( + Arrays.asList( + "mapper_parsing_exception", + "illegal_argument_exception", + "action_request_validation_exception" + ) + ); + + private final boolean logSensitiveData; + protected final AtomicInteger numRecords; + private final AtomicReference error; + protected final BulkProcessor bulkProcessor; + private final ConcurrentMap, SinkRecord> requestToRecord; + private final ConcurrentMap> inFlightRequests; + private final ElasticsearchSinkConnectorConfig config; + private final ErrantRecordReporter reporter; + private final RestHighLevelClient client; + private final ExecutorService bulkExecutorService; + private final Time clock; + + public ElasticsearchClient( + ElasticsearchSinkConnectorConfig config, + ErrantRecordReporter reporter + ) { + this.bulkExecutorService = Executors.newFixedThreadPool(config.maxInFlightRequests()); + this.numRecords = new AtomicInteger(0); + this.error = new AtomicReference<>(); + this.requestToRecord = reporter != null ? new ConcurrentHashMap<>() : null; + this.inFlightRequests = reporter != null ? new ConcurrentHashMap<>() : null; + this.config = config; + this.reporter = reporter; + this.clock = Time.SYSTEM; + this.logSensitiveData = config.shouldLogSensitiveData(); + + ConfigCallbackHandler configCallbackHandler = new ConfigCallbackHandler(config); + this.client = new RestHighLevelClient( + RestClient + .builder( + config.connectionUrls() + .stream() + .map(HttpHost::create) + .collect(Collectors.toList()) + .toArray(new HttpHost[config.connectionUrls().size()]) + ) + .setHttpClientConfigCallback(configCallbackHandler) + ); + this.bulkProcessor = BulkProcessor + .builder(buildConsumer(), buildListener()) + .setBulkActions(config.batchSize()) + .setConcurrentRequests(config.maxInFlightRequests() - 1) // 0 = no concurrent requests + .setFlushInterval(TimeValue.timeValueMillis(config.lingerMs())) + // Disabling bulk processor retries, because they only cover a small subset of errors + // (see https://github.com/elastic/elasticsearch/issues/71159) + // We are doing retries in the async thread instead. + .setBackoffPolicy(BackoffPolicy.noBackoff()) + .build(); + } + + private BiConsumer> buildConsumer() { + return (req, lis) -> + // Executes a synchronous bulk request in a background thread, with synchronous retries. + // We don't use bulkAsync because we can't retry from its callback (see + // https://github.com/confluentinc/kafka-connect-elasticsearch/pull/575) + // BulkProcessor is the one guaranteeing that no more than maxInFlightRequests batches + // are started at the same time (a new consumer is not called until all others are finished), + // which means we don't need to limit the executor pending task queue. + + // Result is ignored because everything is reported via the corresponding ActionListener. + bulkExecutorService.submit(() -> { + try { + BulkResponse bulkResponse = callWithRetries( + "execute bulk request", + () -> client.bulk(req, RequestOptions.DEFAULT) + ); + lis.onResponse(bulkResponse); + } catch (Exception ex) { + lis.onFailure(ex); + } catch (Throwable ex) { + lis.onFailure(new ConnectException("Bulk request failed", ex)); + } + }); } /** - * Gets the Elasticsearch version. + * Returns the underlying Elasticsearch client. * - * @return the version, not null + * @return the underlying RestHighLevelClient */ - Version getVersion(); + public RestHighLevelClient client() { + return client; + } /** - * Creates indices. + * Closes the ElasticsearchClient. * - * @param indices the set of index names to create, not null + * @throws ConnectException if all the records fail to flush before the timeout. */ - void createIndices(Set indices); + public void close() { + try { + if (!bulkProcessor.awaitClose(config.flushTimeoutMs(), TimeUnit.MILLISECONDS)) { + throw new ConnectException( + "Failed to process outstanding requests in time while closing the ElasticsearchClient." + ); + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new ConnectException( + "Interrupted while processing all in-flight requests on ElasticsearchClient close.", e + ); + } finally { + closeResources(); + } + } /** - * Creates an explicit mapping. + * Creates an index. Will not recreate the index if it already exists. * - * @param index the index to write - * @param type the type for which to create the mapping - * @param schema the schema used to infer the mapping - * @throws IOException if the client cannot execute the request + * @param index the index to create + * @return true if the index was created, false if it already exists */ - void createMapping(String index, String type, Schema schema) throws IOException; + public boolean createIndex(String index) { + if (indexExists(index)) { + return false; + } + + CreateIndexRequest request = new CreateIndexRequest(index); + return callWithRetries( + "create index " + index, + () -> { + try { + client.indices().create(request, RequestOptions.DEFAULT); + } catch (ElasticsearchStatusException | IOException e) { + if (!e.getMessage().contains(RESOURCE_ALREADY_EXISTS_EXCEPTION)) { + throw e; + } + return false; + } + return true; + } + ); + } /** - * Gets the JSON mapping for the given index and type. Returns {@code null} if it does not exist. + * Creates a mapping for the given index and schema. * - * @param index the index - * @param type the type - * @throws IOException if the client cannot execute the request + * @param index the index to create the mapping for + * @param schema the schema to map */ - JsonObject getMapping(String index, String type) throws IOException; + public void createMapping(String index, Schema schema) { + PutMappingRequest request = new PutMappingRequest(index).source(Mapping.buildMapping(schema)); + callWithRetries( + String.format("create mapping for index %s with schema %s", index, schema), + () -> client.indices().putMapping(request, RequestOptions.DEFAULT) + ); + } /** - * Creates a bulk request for the list of {@link IndexableRecord} records. + * Flushes any buffered records. + */ + public void flush() { + bulkProcessor.flush(); + } + + /** + * Checks whether the index already has a mapping or not. + * @param index the index to check + * @return true if a mapping exists, false if it does not + */ + public boolean hasMapping(String index) { + MappingMetadata mapping = mapping(index); + return mapping != null && mapping.sourceAsMap() != null && !mapping.sourceAsMap().isEmpty(); + } + + /** + * Buffers a record to index. Will ensure that there are no concurrent requests for the same + * document id when either the DLQ is configured or + * {@link ElasticsearchSinkConnectorConfig#IGNORE_KEY_CONFIG} is set to false because + * they require the use of a map keyed by document id. * - * @param batch the list of records - * @return the bulk request + * @param record the record to index + * @param request the associated request to send + * @throws ConnectException if one of the requests failed */ - BulkRequest createBulkRequest(List batch); + public void index(SinkRecord record, DocWriteRequest request) { + if (isFailed()) { + try { + close(); + } catch (ConnectException e) { + // if close fails, want to still throw the original exception + log.warn("Couldn't close elasticsearch client", e); + } + throw error.get(); + } + + // wait for internal buffer to be less than max.buffered.records configuration + long maxWaitTime = clock.milliseconds() + config.flushTimeoutMs(); + while (numRecords.get() >= config.maxBufferedRecords()) { + clock.sleep(WAIT_TIME_MS); + if (clock.milliseconds() > maxWaitTime) { + throw new ConnectException( + String.format( + "Could not make space in the internal buffer fast enough. Consider increasing %s" + + " or %s.", + FLUSH_TIMEOUT_MS_CONFIG, + MAX_BUFFERED_RECORDS_CONFIG + ) + ); + } + } + + addToRequestToRecordMap(request, record); + numRecords.incrementAndGet(); + bulkProcessor.add(request); + } /** - * Executes a bulk action. + * Checks whether the index exists. * - * @param bulk the bulk request - * @return the bulk response - * @throws IOException if the client cannot execute the request + * @param index the index to check + * @return true if it exists, false if it does not */ - BulkResponse executeBulk(BulkRequest bulk) throws IOException; + public boolean indexExists(String index) { + GetIndexRequest request = new GetIndexRequest(index); + return callWithRetries( + "check if index " + index + " exists", + () -> client.indices().exists(request, RequestOptions.DEFAULT) + ); + } /** - * Executes a search. + * Maps a record to the write request. * - * @param query the search query - * @param index the index to search - * @param type the type to search - * @return the search result - * @throws IOException if the client cannot execute the request + * @param request the write request + * @param record the record */ - JsonObject search(String query, String index, String type) throws IOException; + private void addToRequestToRecordMap(DocWriteRequest request, SinkRecord record) { + if (requestToRecord != null) { + requestToRecord.put(request, record); + } + } /** - * Delete all indexes in Elasticsearch (useful mostly for test) + * Creates a listener with callback functions to handle completed requests for the BulkProcessor. * - * @throws IOException if the client cannot execute the request + * @return the listener */ - default void deleteAll() throws IOException { - throw new UnsupportedOperationException("deleteAll is not implemented yet by this client"); + private BulkProcessor.Listener buildListener() { + return new Listener() { + @Override + public void beforeBulk(long executionId, BulkRequest request) { + if (requestToRecord != null && inFlightRequests != null) { + List sinkRecords = new ArrayList<>(request.requests().size()); + for (DocWriteRequest req : request.requests()) { + sinkRecords.add(requestToRecord.get(req)); + requestToRecord.remove(req); + } + + inFlightRequests.put(executionId, sinkRecords); + } + } + + @Override + public void afterBulk(long executionId, BulkRequest request, BulkResponse response) { + List> requests = request.requests(); + int idx = 0; + for (BulkItemResponse bulkItemResponse : response) { + DocWriteRequest req = idx < requests.size() ? requests.get(idx) : null; + handleResponse(bulkItemResponse, req, executionId); + idx++; + } + removeFromInFlightRequests(executionId); + numRecords.addAndGet(-response.getItems().length); + } + + @Override + public void afterBulk(long executionId, BulkRequest request, Throwable failure) { + log.warn("Bulk request {} failed", executionId, failure); + removeFromInFlightRequests(executionId); + error.compareAndSet(null, new ConnectException("Bulk request failed", failure)); + numRecords.addAndGet(-request.requests().size()); + } + }; } /** - * Perform a refresh of all indexes, making all indexed data searchable (useful mostly for test) + * Returns the formatted error message based on customers need to + * log exception traces. + * @param response The BulkItemResponse returned from Elasticsearch + * @param logSensitiveData Boolean flag to identify if customer needs to + * log exception traces for debugging + * @return String Formatted error message + */ + private String getErrorMessage(BulkItemResponse response, boolean logSensitiveData) { + if (logSensitiveData) { + return response.getFailureMessage(); + } + return String.format("Response status: '%s',\n" + + "Index: '%s',\n Document Id: '%s'. \n", + response.getFailure().getStatus(), + response.getFailure().getIndex(), + response.getFailure().getId()); + } + + /** + * Calls the specified function with retries and backoffs until the retries are exhausted or the + * function succeeds. * - * @throws IOException If the client cannot execute the request + * @param description description of the attempted action in present tense + * @param function the function to call and retry + * @param the return type of the function + * @return the return value of the called function */ - default void refresh() throws IOException { - throw new UnsupportedOperationException("refresh is not implemented yet by this client"); + private T callWithRetries(String description, Callable function) { + return RetryUtil.callWithRetries( + description, + function, + config.maxRetries() + 1, + config.retryBackoffMs() + ); } /** - * Shuts down the client. + * Closes all the connection and thread resources of the client. */ - void close(); + private void closeResources() { + bulkExecutorService.shutdown(); + try { + if (!bulkExecutorService.awaitTermination(CLOSE_WAIT_TIME_MS, TimeUnit.MILLISECONDS)) { + bulkExecutorService.shutdownNow(); + } + } catch (InterruptedException e) { + bulkExecutorService.shutdownNow(); + Thread.currentThread().interrupt(); + log.warn("Interrupted while awaiting for executor service shutdown.", e); + } + + try { + client.close(); + } catch (IOException e) { + log.warn("Failed to close Elasticsearch client.", e); + } + } + + /** + * Processes a response from a {@link org.elasticsearch.action.bulk.BulkItemRequest}. + * Successful responses are ignored. Failed responses are reported to the DLQ and handled + * according to configuration (ignore or fail). Version conflicts are ignored. + * + * @param response the response to process + * @param request the request which generated the response + * @param executionId the execution id of the request + */ + protected void handleResponse(BulkItemResponse response, DocWriteRequest request, + long executionId) { + if (response.isFailed()) { + for (String error : MALFORMED_DOC_ERRORS) { + if (response.getFailureMessage().contains(error)) { + handleMalformedDocResponse(response); + reportBadRecord(response, executionId); + return; + } + } + if (response.getFailureMessage().contains(VERSION_CONFLICT_EXCEPTION)) { + // Now check if this version conflict is caused by external version number + // which was set by us (set explicitly to the topic's offset), in which case + // the version conflict is due to a repeated or out-of-order message offset + // and thus can be ignored, since the newer value (higher offset) should + // remain the key's value in any case. + if (request == null || request.versionType() != VersionType.EXTERNAL) { + log.warn("{} version conflict for operation {} on document '{}' version {}" + + " in index '{}'.", + request != null ? request.versionType() : "UNKNOWN", + response.getOpType(), + response.getId(), + response.getVersion(), + response.getIndex() + ); + // Maybe this was a race condition? Put it in the DLQ in case someone + // wishes to investigate. + reportBadRecord(response, executionId); + } else { + // This is an out-of-order or (more likely) repeated topic offset. Allow the + // higher offset's value for this key to remain. + // + // Note: For external version conflicts, response.getVersion() will be returned as -1, + // but we have the actual version number for this record because we set it in + // the request. + log.debug("Ignoring EXTERNAL version conflict for operation {} on" + + " document '{}' version {} in index '{}'.", + response.getOpType(), + response.getId(), + request.version(), + response.getIndex() + ); + } + return; + } + + error.compareAndSet( + null, + new ConnectException("Indexing record failed.", + new Throwable(getErrorMessage(response, logSensitiveData))) + ); + } + } + + /** + * Handle a failed response as a result of a malformed document. Depending on the configuration, + * ignore or fail. + * + * @param response the failed response from ES + */ + private void handleMalformedDocResponse(BulkItemResponse response) { + String errorMsg = String.format("Encountered an illegal document error '%s'." + + " Ignoring and will not index record." , getErrorMessage(response, logSensitiveData)); + switch (config.behaviorOnMalformedDoc()) { + case IGNORE: + log.debug(errorMsg); + return; + case WARN: + log.warn(errorMsg); + return; + case FAIL: + default: + log.error(String.format("Encountered an illegal document error '%s'." + + " To ignore future records like this," + + " change the configuration '%s' to '%s'.", + getErrorMessage(response, logSensitiveData), + ElasticsearchSinkConnectorConfig.BEHAVIOR_ON_MALFORMED_DOCS_CONFIG, + BehaviorOnMalformedDoc.IGNORE) + ); + error.compareAndSet( + null, + new ConnectException( + "Indexing record failed -> " + getErrorMessage(response, logSensitiveData)) + ); + } + } + + /** + * Whether there is a failed response. + * + * @return true if a response has failed, false if none have failed + */ + private boolean isFailed() { + return error.get() != null; + } + + /** + * Gets the mapping for an index. + * + * @param index the index to fetch the mapping for + * @return the MappingMetadata for the index + */ + private MappingMetadata mapping(String index) { + GetMappingsRequest request = new GetMappingsRequest().indices(index); + GetMappingsResponse response = callWithRetries( + "get mapping for index " + index, + () -> client.indices().getMapping(request, RequestOptions.DEFAULT) + ); + return response.mappings().get(index); + } + + /** + * Removes the mapping for bulk request id to records being written. + * + * @param executionDd the execution id of the bulk request + */ + private void removeFromInFlightRequests(long executionDd) { + if (inFlightRequests != null) { + inFlightRequests.remove(executionDd); + } + } + + /** + * Reports a bad record to the DLQ. + * + * @param response the failed response from ES + * @param executionId the execution id of the request associated with the response + */ + private synchronized void reportBadRecord(BulkItemResponse response, long executionId) { + if (reporter != null) { + List sinkRecords = inFlightRequests.getOrDefault(executionId, new ArrayList<>()); + SinkRecord original = sinkRecords.size() > response.getItemId() + ? sinkRecords.get(response.getItemId()) + : null; + if (original != null) { + reporter.report( + original, + new ReportingException("Indexing failed: " + getErrorMessage(response,logSensitiveData)) + ); + } + } + } + + /** + * Exception that swallows the stack trace used for reporting errors from Elasticsearch + * (mapper_parser_exception, illegal_argument_exception, and action_request_validation_exception) + * resulting from bad records using the AK 2.6 reporter DLQ interface. + */ + @SuppressWarnings("serial") + public static class ReportingException extends RuntimeException { + + public ReportingException(String message) { + super(message); + } + + /** + * This method is overridden to swallow the stack trace. + * + * @return Throwable + */ + @Override + public synchronized Throwable fillInStackTrace() { + return this; + } + } } diff --git a/src/main/java/io/confluent/connect/elasticsearch/ElasticsearchSinkConnector.java b/src/main/java/io/confluent/connect/elasticsearch/ElasticsearchSinkConnector.java index 00443f8c0..070a95037 100644 --- a/src/main/java/io/confluent/connect/elasticsearch/ElasticsearchSinkConnector.java +++ b/src/main/java/io/confluent/connect/elasticsearch/ElasticsearchSinkConnector.java @@ -67,9 +67,7 @@ public List> taskConfigs(int maxTasks) { } @Override - public void stop() throws ConnectException { - - } + public void stop() throws ConnectException { } @Override public ConfigDef config() { diff --git a/src/main/java/io/confluent/connect/elasticsearch/ElasticsearchSinkConnectorConfig.java b/src/main/java/io/confluent/connect/elasticsearch/ElasticsearchSinkConnectorConfig.java index 28d226215..01df9dfea 100644 --- a/src/main/java/io/confluent/connect/elasticsearch/ElasticsearchSinkConnectorConfig.java +++ b/src/main/java/io/confluent/connect/elasticsearch/ElasticsearchSinkConnectorConfig.java @@ -15,30 +15,24 @@ package io.confluent.connect.elasticsearch; +import java.io.File; import java.net.URI; import java.net.URISyntaxException; -import java.util.Arrays; -import java.util.HashMap; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Set; -import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; +import java.util.concurrent.TimeUnit; import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; -import org.apache.kafka.common.config.ConfigDef.CaseInsensitiveValidString; +import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.config.ConfigDef.Importance; import org.apache.kafka.common.config.ConfigDef.Type; import org.apache.kafka.common.config.ConfigDef.Validator; import org.apache.kafka.common.config.ConfigDef.Width; - -import java.util.Map; -import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.config.types.Password; -import static io.confluent.connect.elasticsearch.jest.JestElasticsearchClient.WriteMethod; -import static io.confluent.connect.elasticsearch.DataConverter.BehaviorOnNullValues; -import static io.confluent.connect.elasticsearch.bulk.BulkProcessor.BehaviorOnMalformedDoc; import static org.apache.kafka.common.config.ConfigDef.Range.between; import static org.apache.kafka.common.config.SslConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG; import static org.apache.kafka.common.config.SslConfigs.addClientSslSupport; @@ -106,11 +100,11 @@ public class ElasticsearchSinkConnectorConfig extends AbstractConfig { public static final String FLUSH_TIMEOUT_MS_CONFIG = "flush.timeout.ms"; private static final String FLUSH_TIMEOUT_MS_DOC = - "The timeout in milliseconds to use for periodic flushing, and when waiting for buffer " - + "space to be made available by completed requests as records are added. If this timeout " - + "is exceeded the task will fail."; + "The timeout in milliseconds to use for periodic flushing, and when waiting for buffer space" + + " to be made available by completed requests as records are added. If this timeout is" + + " exceeded the task will fail."; private static final String FLUSH_TIMEOUT_MS_DISPLAY = "Flush Timeout (ms)"; - private static final int FLUSH_TIMEOUT_MS_DEFAULT = (int) TimeUnit.SECONDS.toMillis(10); + private static final int FLUSH_TIMEOUT_MS_DEFAULT = (int) TimeUnit.MINUTES.toMillis(3); public static final String MAX_RETRIES_CONFIG = "max.retries"; private static final String MAX_RETRIES_DOC = @@ -142,7 +136,7 @@ public class ElasticsearchSinkConnectorConfig extends AbstractConfig { + "in milliseconds before dropping an idle connection to prevent " + "a read timeout."; private static final String MAX_CONNECTION_IDLE_TIME_MS_DISPLAY = "Max Connection Idle Time"; - private static final int MAX_CONNECTION_IDLE_TIME_MS_DEFAULT = 60000; + private static final int MAX_CONNECTION_IDLE_TIME_MS_DEFAULT = (int) TimeUnit.MINUTES.toMillis(1); public static final String CONNECTION_TIMEOUT_MS_CONFIG = "connection.timeout.ms"; private static final String CONNECTION_TIMEOUT_MS_CONFIG_DOC = "How long to wait " @@ -160,18 +154,7 @@ public class ElasticsearchSinkConnectorConfig extends AbstractConfig { private static final String READ_TIMEOUT_MS_DISPLAY = "Read Timeout"; private static final int READ_TIMEOUT_MS_DEFAULT = (int) TimeUnit.SECONDS.toMillis(3); - public static final String CREATE_INDICES_AT_START_CONFIG = "auto.create.indices.at.start"; - private static final String CREATE_INDICES_AT_START_DOC = "Auto create the Elasticsearch" - + " indices at startup. This is useful when the indices are a direct mapping " - + " of the Kafka topics."; - private static final String CREATE_INDICES_AT_START_DISPLAY = "Create indices at startup"; - private static final boolean CREATE_INDICES_AT_START_DEFAULT = true; - // Data Conversion configs - public static final String TYPE_NAME_CONFIG = "type.name"; - private static final String TYPE_NAME_DOC = "The Elasticsearch type name to use when indexing."; - private static final String TYPE_NAME_DISPLAY = "Type Name"; - public static final String IGNORE_KEY_TOPICS_CONFIG = "topic.key.ignore"; public static final String IGNORE_SCHEMA_TOPICS_CONFIG = "topic.schema.ignore"; @@ -209,16 +192,6 @@ public class ElasticsearchSinkConnectorConfig extends AbstractConfig { private static final String COMPACT_MAP_ENTRIES_DISPLAY = "Compact Map Entries"; private static final boolean COMPACT_MAP_ENTRIES_DEFAULT = true; - @Deprecated - public static final String TOPIC_INDEX_MAP_CONFIG = "topic.index.map"; - private static final String TOPIC_INDEX_MAP_DOC = - "This option is now deprecated. A future version may remove it completely. Please use " - + "single message transforms, such as RegexRouter, to map topic names to index names.\n" - + "A map from Kafka topic name to the destination Elasticsearch index, represented as " - + "a list of ``topic:index`` pairs."; - private static final String TOPIC_INDEX_MAP_DISPLAY = "Topic to Index Map"; - private static final String TOPIC_INDEX_MAP_DEFAULT = ""; - private static final String IGNORE_KEY_TOPICS_DOC = "List of topics for which ``" + IGNORE_KEY_CONFIG + "`` should be ``true``."; private static final String IGNORE_KEY_TOPICS_DISPLAY = "Topics for 'Ignore Key' mode"; @@ -240,6 +213,8 @@ public class ElasticsearchSinkConnectorConfig extends AbstractConfig { + "non-null key and a null value (i.e. Kafka tombstone records). Valid options are " + "'ignore', 'delete', and 'fail'."; private static final String BEHAVIOR_ON_NULL_VALUES_DISPLAY = "Behavior for null-valued records"; + private static final BehaviorOnNullValues BEHAVIOR_ON_NULL_VALUES_DEFAULT = + BehaviorOnNullValues.FAIL; public static final String BEHAVIOR_ON_MALFORMED_DOCS_CONFIG = "behavior.on.malformed.documents"; private static final String BEHAVIOR_ON_MALFORMED_DOCS_DOC = "How to handle records that " @@ -248,21 +223,43 @@ public class ElasticsearchSinkConnectorConfig extends AbstractConfig { + " id. Valid options are ignore', 'warn', and 'fail'."; private static final String BEHAVIOR_ON_MALFORMED_DOCS_DISPLAY = "Behavior on malformed documents"; + private static final BehaviorOnMalformedDoc BEHAVIOR_ON_MALFORMED_DOCS_DEFAULT = + BehaviorOnMalformedDoc.FAIL; + + public static final String EXTERNAL_VERSION_HEADER_CONFIG = "external.version.header"; + private static final String EXTERNAL_VERSION_HEADER_DOC = "Header name to pull value for" + + " external versioning, defaults to using the kafka record offset. Must have a numeric" + + " value."; + private static final String EXTERNAL_VERSION_HEADER_DISPLAY = "External Version Header Name"; + private static final String EXTERNAL_VERSION_HEADER_DEFAULT = ""; public static final String WRITE_METHOD_CONFIG = "write.method"; - private static final String WRITE_METHOD_DOC = "Method used for writing data to Elasticsearch," - + " and one of " + WriteMethod.INSERT.toString() + " or " + WriteMethod.UPSERT.toString() - + ". The default method is " + WriteMethod.INSERT.toString() + ", in which the " - + "connector constructs a document from the record value and inserts that document " - + "into Elasticsearch, completely replacing any existing document with the same ID; " - + "this matches previous behavior. The " + WriteMethod.UPSERT.toString() - + " method will create a new document if one with the specified ID does not yet " - + "exist, or will update an existing document with the same ID by adding/replacing " - + "only those fields present in the record value. The " + WriteMethod.UPSERT.toString() - + " method may require additional time and resources of Elasticsearch, so consider " - + "increasing the " + FLUSH_TIMEOUT_MS_CONFIG + ", " + READ_TIMEOUT_MS_CONFIG - + ", and decrease " + BATCH_SIZE_CONFIG + " configuration properties."; + private static final String WRITE_METHOD_DOC = String.format( + "Method used for writing data to Elasticsearch, and one of %s or %s. The default method is" + + " %s, in which the connector constructs a document from the record value and inserts" + + " that document into Elasticsearch, completely replacing any existing document with the" + + " same ID; this matches previous behavior. The %s method will create a new document if" + + " one with the specified ID does not yet exist, or will update an existing document" + + " with the same ID by adding/replacing only those fields present in the record value." + + " The %s method may require additional time and resources of Elasticsearch, so consider" + + " increasing the %s and decreasing the %s configuration properties.", + WriteMethod.INSERT, + WriteMethod.UPSERT, + WriteMethod.INSERT, + WriteMethod.UPSERT, + WriteMethod.UPSERT, + READ_TIMEOUT_MS_CONFIG, + BATCH_SIZE_CONFIG + ); private static final String WRITE_METHOD_DISPLAY = "Write Method"; + private static final String WRITE_METHOD_DEFAULT = WriteMethod.INSERT.name(); + public static final String LOG_SENSITIVE_DATA_CONFIG = "log.sensitive.data"; + private static final String LOG_SENSITIVE_DATA_DISPLAY = "Log Sensitive data"; + private static final String LOG_SENSITIVE_DATA_DOC = "If true, logs sensitive data " + + "(such as exception traces containing sensitive data). " + + "Set this to true only in exceptional scenarios where logging sensitive data " + + "is acceptable and is necessary for troubleshooting."; + private static final Boolean LOG_SENSITIVE_DATA_DEFAULT = Boolean.FALSE; // Proxy group public static final String PROXY_HOST_CONFIG = "proxy.host"; @@ -291,23 +288,60 @@ public class ElasticsearchSinkConnectorConfig extends AbstractConfig { public static final String SECURITY_PROTOCOL_CONFIG = "elastic.security.protocol"; private static final String SECURITY_PROTOCOL_DOC = - "The security protocol to use when connecting to Elasticsearch. " - + "Values can be `PLAINTEXT` or `SSL`. If `PLAINTEXT` is passed, " - + "all configs prefixed by " + SSL_CONFIG_PREFIX + " will be ignored."; + "The security protocol to use when connecting to Elasticsearch. Values can be `PLAINTEXT` or" + + " `SSL`. If `PLAINTEXT` is passed, all configs prefixed by " + SSL_CONFIG_PREFIX + + " will be ignored."; private static final String SECURITY_PROTOCOL_DISPLAY = "Security protocol"; private static final String SECURITY_PROTOCOL_DEFAULT = SecurityProtocol.PLAINTEXT.name(); + // Kerberos configs + public static final String KERBEROS_PRINCIPAL_CONFIG = "kerberos.user.principal"; + private static final String KERBEROS_PRINCIPAL_DISPLAY = "Kerberos User Principal"; + private static final String KERBEROS_PRINCIPAL_DOC = "The Kerberos user principal the connector" + + " may use to authenticate with Kerberos."; + private static final String KERBEROS_PRINCIPAL_DEFAULT = null; + + public static final String KERBEROS_KEYTAB_PATH_CONFIG = "kerberos.keytab.path"; + private static final String KERBEROS_KEYTAB_PATH = "Kerberos Keytab File Path"; + private static final String KERBEROS_KEYTAB_PATH_DOC = "The path to the keytab file to use for" + + " authentication with Kerberos."; + private static final String KERBEROS_KEYTAB_PATH_DEFAULT = null; + private static final String CONNECTOR_GROUP = "Connector"; private static final String DATA_CONVERSION_GROUP = "Data Conversion"; private static final String PROXY_GROUP = "Proxy"; private static final String SSL_GROUP = "Security"; + private static final String KERBEROS_GROUP = "Kerberos"; + + public enum BehaviorOnMalformedDoc { + IGNORE, + WARN, + FAIL + } + + public enum BehaviorOnNullValues { + IGNORE, + DELETE, + FAIL + } + + public enum SecurityProtocol { + PLAINTEXT, + SSL + } + + public enum WriteMethod { + INSERT, + UPSERT + } protected static ConfigDef baseConfigDef() { final ConfigDef configDef = new ConfigDef(); addConnectorConfigs(configDef); addConversionConfigs(configDef); addProxyConfigs(configDef); - addSecurityConfigs(configDef); + addSslConfigs(configDef); + addKerberosConfigs(configDef); return configDef; } @@ -393,7 +427,7 @@ private static void addConnectorConfigs(ConfigDef configDef) { FLUSH_TIMEOUT_MS_CONFIG, Type.LONG, FLUSH_TIMEOUT_MS_DEFAULT, - between(TimeUnit.SECONDS.toMillis(1), Long.MAX_VALUE), + between(TimeUnit.SECONDS.toMillis(1), TimeUnit.DAYS.toMillis(7)), Importance.LOW, FLUSH_TIMEOUT_MS_DOC, CONNECTOR_GROUP, @@ -436,6 +470,7 @@ private static void addConnectorConfigs(ConfigDef configDef) { MAX_CONNECTION_IDLE_TIME_MS_CONFIG, Type.INT, MAX_CONNECTION_IDLE_TIME_MS_DEFAULT, + between(-1, TimeUnit.DAYS.toMillis(1)), Importance.LOW, MAX_CONNECTION_IDLE_TIME_MS_CONFIG_DOC, CONNECTOR_GROUP, @@ -465,15 +500,15 @@ private static void addConnectorConfigs(ConfigDef configDef) { Width.SHORT, READ_TIMEOUT_MS_DISPLAY ).define( - CREATE_INDICES_AT_START_CONFIG, + LOG_SENSITIVE_DATA_CONFIG, Type.BOOLEAN, - CREATE_INDICES_AT_START_DEFAULT, + LOG_SENSITIVE_DATA_DEFAULT, Importance.LOW, - CREATE_INDICES_AT_START_DOC, + LOG_SENSITIVE_DATA_DOC, CONNECTOR_GROUP, ++order, Width.SHORT, - CREATE_INDICES_AT_START_DISPLAY + LOG_SENSITIVE_DATA_DISPLAY ); } @@ -481,15 +516,6 @@ private static void addConversionConfigs(ConfigDef configDef) { int order = 0; configDef .define( - TYPE_NAME_CONFIG, - Type.STRING, - Importance.HIGH, - TYPE_NAME_DOC, - DATA_CONVERSION_GROUP, - ++order, - Width.SHORT, - TYPE_NAME_DISPLAY - ).define( IGNORE_KEY_CONFIG, Type.BOOLEAN, IGNORE_KEY_DEFAULT, @@ -519,16 +545,6 @@ private static void addConversionConfigs(ConfigDef configDef) { ++order, Width.SHORT, COMPACT_MAP_ENTRIES_DISPLAY - ).define( - TOPIC_INDEX_MAP_CONFIG, - Type.LIST, - TOPIC_INDEX_MAP_DEFAULT, - Importance.LOW, - TOPIC_INDEX_MAP_DOC, - DATA_CONVERSION_GROUP, - ++order, - Width.LONG, - TOPIC_INDEX_MAP_DISPLAY ).define( IGNORE_KEY_TOPICS_CONFIG, Type.LIST, @@ -562,36 +578,49 @@ private static void addConversionConfigs(ConfigDef configDef) { ).define( BEHAVIOR_ON_NULL_VALUES_CONFIG, Type.STRING, - BehaviorOnNullValues.DEFAULT.toString(), - BehaviorOnNullValues.VALIDATOR, + BEHAVIOR_ON_NULL_VALUES_DEFAULT.name(), + new EnumRecommender<>(BehaviorOnNullValues.class), Importance.LOW, BEHAVIOR_ON_NULL_VALUES_DOC, DATA_CONVERSION_GROUP, ++order, Width.SHORT, - BEHAVIOR_ON_NULL_VALUES_DISPLAY + BEHAVIOR_ON_NULL_VALUES_DISPLAY, + new EnumRecommender<>(BehaviorOnNullValues.class) ).define( BEHAVIOR_ON_MALFORMED_DOCS_CONFIG, Type.STRING, - BehaviorOnMalformedDoc.DEFAULT.toString(), - BehaviorOnMalformedDoc.VALIDATOR, + BEHAVIOR_ON_MALFORMED_DOCS_DEFAULT.name(), + new EnumRecommender<>(BehaviorOnMalformedDoc.class), Importance.LOW, BEHAVIOR_ON_MALFORMED_DOCS_DOC, DATA_CONVERSION_GROUP, ++order, Width.SHORT, - BEHAVIOR_ON_MALFORMED_DOCS_DISPLAY + BEHAVIOR_ON_MALFORMED_DOCS_DISPLAY, + new EnumRecommender<>(BehaviorOnMalformedDoc.class) + ).define( + EXTERNAL_VERSION_HEADER_CONFIG, + Type.STRING, + EXTERNAL_VERSION_HEADER_DEFAULT, + Importance.LOW, + EXTERNAL_VERSION_HEADER_DOC, + DATA_CONVERSION_GROUP, + ++order, + Width.SHORT, + EXTERNAL_VERSION_HEADER_DISPLAY ).define( WRITE_METHOD_CONFIG, Type.STRING, - WriteMethod.DEFAULT.toString(), - WriteMethod.VALIDATOR, + WRITE_METHOD_DEFAULT, + new EnumRecommender<>(WriteMethod.class), Importance.LOW, WRITE_METHOD_DOC, DATA_CONVERSION_GROUP, ++order, Width.SHORT, - WRITE_METHOD_DISPLAY + WRITE_METHOD_DISPLAY, + new EnumRecommender<>(WriteMethod.class) ); } @@ -642,7 +671,7 @@ private static void addProxyConfigs(ConfigDef configDef) { ); } - private static void addSecurityConfigs(ConfigDef configDef) { + private static void addSslConfigs(ConfigDef configDef) { ConfigDef sslConfigDef = new ConfigDef(); addClientSslSupport(sslConfigDef); int order = 0; @@ -650,21 +679,42 @@ private static void addSecurityConfigs(ConfigDef configDef) { SECURITY_PROTOCOL_CONFIG, Type.STRING, SECURITY_PROTOCOL_DEFAULT, - CaseInsensitiveValidString.in( - Arrays.stream(SecurityProtocol.values()) - .map(SecurityProtocol::name) - .collect(Collectors.toList()) - .toArray(new String[SecurityProtocol.values().length]) - ), + new EnumRecommender<>(SecurityProtocol.class), Importance.MEDIUM, SECURITY_PROTOCOL_DOC, SSL_GROUP, ++order, Width.SHORT, - SECURITY_PROTOCOL_DISPLAY + SECURITY_PROTOCOL_DISPLAY, + new EnumRecommender<>(SecurityProtocol.class) ); - configDef.embed( - SSL_CONFIG_PREFIX, SSL_GROUP, configDef.configKeys().size() + 2, sslConfigDef + configDef.embed(SSL_CONFIG_PREFIX, SSL_GROUP, configDef.configKeys().size() + 2, sslConfigDef); + } + + private static void addKerberosConfigs(ConfigDef configDef) { + int orderInGroup = 0; + configDef + .define( + KERBEROS_PRINCIPAL_CONFIG, + Type.STRING, + KERBEROS_PRINCIPAL_DEFAULT, + Importance.LOW, + KERBEROS_PRINCIPAL_DOC, + KERBEROS_GROUP, + orderInGroup++, + Width.LONG, + KERBEROS_PRINCIPAL_DISPLAY + ).define( + KERBEROS_KEYTAB_PATH_CONFIG, + Type.STRING, + KERBEROS_KEYTAB_PATH_DEFAULT, + new FilePathValidator("keytab"), + Importance.LOW, + KERBEROS_KEYTAB_PATH_DOC, + KERBEROS_GROUP, + orderInGroup++, + Width.LONG, + KERBEROS_KEYTAB_PATH ); } @@ -688,9 +738,12 @@ public boolean isProxyWithAuthenticationConfigured() { && getPassword(PROXY_PASSWORD_CONFIG) != null; } - public boolean secured() { - SecurityProtocol securityProtocol = securityProtocol(); - return SecurityProtocol.SSL.equals(securityProtocol); + public boolean isKerberosEnabled() { + return kerberosUserPrincipal() != null || keytabPath() != null; + } + + public boolean isSslEnabled() { + return SecurityProtocol.SSL == securityProtocol(); } public boolean shouldDisableHostnameVerification() { @@ -700,6 +753,14 @@ public boolean shouldDisableHostnameVerification() { && sslEndpointIdentificationAlgorithm.isEmpty(); } + public boolean shouldIgnoreKey(String topic) { + return ignoreKey() || ignoreKeyTopics().contains(topic); + } + + public boolean shouldIgnoreSchema(String topic) { + return ignoreSchema() || ignoreSchemaTopics().contains(topic); + } + public int batchSize() { return getInt(BATCH_SIZE_CONFIG); } @@ -718,16 +779,18 @@ public boolean compression() { return getBoolean(CONNECTION_COMPRESSION_CONFIG); } + public boolean shouldLogSensitiveData() { + return getBoolean(LOG_SENSITIVE_DATA_CONFIG); + } + public int connectionTimeoutMs() { return getInt(CONNECTION_TIMEOUT_MS_CONFIG); } public Set connectionUrls() { - return new HashSet<>(getList(CONNECTION_URL_CONFIG)); - } - - public boolean createIndicesAtStart() { - return getBoolean(CREATE_INDICES_AT_START_CONFIG); + return getList(CONNECTION_URL_CONFIG) + .stream().map(s -> s.endsWith("/") ? s.substring(0, s.length() - 1) : s) + .collect(Collectors.toCollection(HashSet::new)); } public boolean dropInvalidMessage() { @@ -754,6 +817,14 @@ public Set ignoreSchemaTopics() { return new HashSet<>(getList(IGNORE_SCHEMA_TOPICS_CONFIG)); } + public String kerberosUserPrincipal() { + return getString(KERBEROS_PRINCIPAL_CONFIG); + } + + public String keytabPath() { + return getString(KERBEROS_KEYTAB_PATH_CONFIG); + } + public long lingerMs() { return getLong(LINGER_MS_CONFIG); } @@ -803,7 +874,7 @@ public long retryBackoffMs() { } private SecurityProtocol securityProtocol() { - return SecurityProtocol.valueOf(getString(SECURITY_PROTOCOL_CONFIG)); + return SecurityProtocol.valueOf(getString(SECURITY_PROTOCOL_CONFIG).toUpperCase()); } public Map sslConfigs() { @@ -812,15 +883,6 @@ public Map sslConfigs() { return sslConfigDef.parse(originalsWithPrefix(SSL_CONFIG_PREFIX)); } - @Deprecated - public Map topicToIndexMap() { - return parseMapConfig(getList(TOPIC_INDEX_MAP_CONFIG)); - } - - public String type() { - return getString(TYPE_NAME_CONFIG); - } - public String username() { return getString(CONNECTION_USERNAME_CONFIG); } @@ -829,20 +891,16 @@ public boolean useCompactMapEntries() { return getBoolean(COMPACT_MAP_ENTRIES_CONFIG); } - public WriteMethod writeMethod() { - return WriteMethod.valueOf(getString(WRITE_METHOD_CONFIG).toUpperCase()); + public boolean hasExternalVersionHeader() { + return !getString(EXTERNAL_VERSION_HEADER_CONFIG).isEmpty(); } - private Map parseMapConfig(List values) { - Map map = new HashMap<>(); - for (String value : values) { - String[] parts = value.split(":"); - String topic = parts[0]; - String type = parts[1]; - map.put(topic, type); - } + public String externalVersionHeader() { + return getString(EXTERNAL_VERSION_HEADER_CONFIG); + } - return map; + public WriteMethod writeMethod() { + return WriteMethod.valueOf(getString(WRITE_METHOD_CONFIG).toUpperCase()); } private static class UrlListValidator implements Validator { @@ -851,10 +909,9 @@ private static class UrlListValidator implements Validator { @SuppressWarnings("unchecked") public void ensureValid(String name, Object value) { if (value == null) { - throw new ConfigException( - name, value, "The provided url list is null." - ); + throw new ConfigException(name, value, "The config must be provided and non-null."); } + List urls = (List) value; for (String url : urls) { try { @@ -873,6 +930,37 @@ public String toString() { } } + private static class FilePathValidator implements Validator { + + private String extension; + + public FilePathValidator(String extension) { + this.extension = extension; + } + + @Override + @SuppressWarnings("unchecked") + public void ensureValid(String name, Object value) { + if (value == null) { + return; + } + + if (!((String) value).endsWith(extension)) { + throw new ConfigException(name, value, "The specified file must end with ." + extension); + } + + File file = new File((String) value); + if (!file.exists()) { + throw new ConfigException(name, value, "The specified file does not exist."); + } + } + + @Override + public String toString() { + return "Existing file with " + extension + " extension."; + } + } + public static void main(String[] args) { System.out.println(CONFIG.toEnrichedRst()); } diff --git a/src/main/java/io/confluent/connect/elasticsearch/ElasticsearchSinkConnectorConstants.java b/src/main/java/io/confluent/connect/elasticsearch/ElasticsearchSinkConnectorConstants.java deleted file mode 100644 index de7bd8231..000000000 --- a/src/main/java/io/confluent/connect/elasticsearch/ElasticsearchSinkConnectorConstants.java +++ /dev/null @@ -1,34 +0,0 @@ -/* - * 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.connect.elasticsearch; - -public class ElasticsearchSinkConnectorConstants { - public static final String MAP_KEY = "key"; - public static final String MAP_VALUE = "value"; - - public static final String BOOLEAN_TYPE = "boolean"; - public static final String BYTE_TYPE = "byte"; - public static final String BINARY_TYPE = "binary"; - public static final String SHORT_TYPE = "short"; - public static final String INTEGER_TYPE = "integer"; - public static final String LONG_TYPE = "long"; - public static final String FLOAT_TYPE = "float"; - public static final String DOUBLE_TYPE = "double"; - public static final String STRING_TYPE = "string"; - public static final String TEXT_TYPE = "text"; - public static final String KEYWORD_TYPE = "keyword"; - public static final String DATE_TYPE = "date"; -} diff --git a/src/main/java/io/confluent/connect/elasticsearch/ElasticsearchSinkTask.java b/src/main/java/io/confluent/connect/elasticsearch/ElasticsearchSinkTask.java index 90fe50d35..936bc0172 100644 --- a/src/main/java/io/confluent/connect/elasticsearch/ElasticsearchSinkTask.java +++ b/src/main/java/io/confluent/connect/elasticsearch/ElasticsearchSinkTask.java @@ -15,13 +15,19 @@ package io.confluent.connect.elasticsearch; -import io.confluent.connect.elasticsearch.jest.JestElasticsearchClient; +import org.apache.http.HttpHost; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.connect.errors.ConnectException; +import org.apache.kafka.connect.errors.DataException; +import org.apache.kafka.connect.sink.ErrantRecordReporter; import org.apache.kafka.connect.sink.SinkRecord; import org.apache.kafka.connect.sink.SinkTask; +import org.elasticsearch.action.DocWriteRequest; +import org.elasticsearch.client.RequestOptions; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.RestHighLevelClient; +import org.elasticsearch.client.core.MainResponse; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -29,139 +35,218 @@ import java.util.HashSet; import java.util.Map; import java.util.Set; -import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +import io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.BehaviorOnNullValues; public class ElasticsearchSinkTask extends SinkTask { private static final Logger log = LoggerFactory.getLogger(ElasticsearchSinkTask.class); - private volatile ElasticsearchWriter writer; - private ElasticsearchClient client; - private Boolean createIndicesAtStartTime; - @Override - public String version() { - return Version.getVersion(); - } + private DataConverter converter; + private ElasticsearchClient client; + private ElasticsearchSinkConnectorConfig config; + private ErrantRecordReporter reporter; + private Set existingMappings; + private Set indexCache; @Override public void start(Map props) { start(props, null); } - @SuppressWarnings("deprecation") - // public for testing - public void start(Map props, ElasticsearchClient client) { + // visible for testing + protected void start(Map props, ElasticsearchClient client) { + log.info("Starting ElasticsearchSinkTask."); + + this.config = new ElasticsearchSinkConnectorConfig(props); + this.converter = new DataConverter(config); + this.existingMappings = new HashSet<>(); + this.indexCache = new HashSet<>(); + + this.reporter = null; try { - log.info("Starting ElasticsearchSinkTask"); - - ElasticsearchSinkConnectorConfig config = new ElasticsearchSinkConnectorConfig(props); - - // Calculate the maximum possible backoff time ... - long maxRetryBackoffMs = - RetryUtil.computeRetryWaitTimeInMillis(config.maxRetries(), config.retryBackoffMs()); - if (maxRetryBackoffMs > RetryUtil.MAX_RETRY_TIME_MS) { - log.warn("This connector uses exponential backoff with jitter for retries, " - + "and using '{}={}' and '{}={}' results in an impractical but possible maximum " - + "backoff time greater than {} hours.", - ElasticsearchSinkConnectorConfig.MAX_RETRIES_CONFIG, config.maxRetries(), - ElasticsearchSinkConnectorConfig.RETRY_BACKOFF_MS_CONFIG, config.retryBackoffMs(), - TimeUnit.MILLISECONDS.toHours(maxRetryBackoffMs)); + if (context.errantRecordReporter() == null) { + log.info("Errant record reporter not configured."); } + // may be null if DLQ not enabled + reporter = context.errantRecordReporter(); + } catch (NoClassDefFoundError | NoSuchMethodError e) { + // Will occur in Connect runtimes earlier than 2.6 + log.warn("AK versions prior to 2.6 do not support the errant record reporter."); + } - if (client != null) { - this.client = client; - } else { - this.client = new JestElasticsearchClient(props); - } + this.client = client != null ? client : new ElasticsearchClient(config, reporter); - ElasticsearchWriter.Builder builder = new ElasticsearchWriter.Builder(this.client) - .setType(config.type()) - .setIgnoreKey(config.ignoreKey(), config.ignoreKeyTopics()) - .setIgnoreSchema(config.ignoreSchema(), config.ignoreSchemaTopics()) - .setCompactMapEntries(config.useCompactMapEntries()) - .setTopicToIndexMap(config.topicToIndexMap()) - .setFlushTimoutMs(config.flushTimeoutMs()) - .setMaxBufferedRecords(config.maxBufferedRecords()) - .setMaxInFlightRequests(config.maxInFlightRequests()) - .setBatchSize(config.batchSize()) - .setLingerMs(config.lingerMs()) - .setRetryBackoffMs(config.retryBackoffMs()) - .setMaxRetry(config.maxRetries()) - .setDropInvalidMessage(config.dropInvalidMessage()) - .setBehaviorOnNullValues(config.behaviorOnNullValues()) - .setBehaviorOnMalformedDoc(config.behaviorOnMalformedDoc()); + log.info("Started ElasticsearchSinkTask. Connecting to ES server version: {}", + getServerVersion()); + } - try { - if (context.errantRecordReporter() == null) { - log.info("Errant record reporter not configured."); - } - - // may be null if DLQ not enabled - builder.setErrantRecordReporter(context.errantRecordReporter()); - } catch (NoClassDefFoundError | NoSuchMethodError e) { - // Will occur in Connect runtimes earlier than 2.6 - log.warn("AK versions prior to 2.6 do not support the errant record reporter"); + @Override + public void put(Collection records) throws ConnectException { + log.debug("Putting {} records to Elasticsearch.", records.size()); + for (SinkRecord record : records) { + if (shouldSkipRecord(record)) { + logTrace("Ignoring {} with null value.", record); + reportBadRecord(record, new ConnectException("Cannot write null valued record.")); + continue; } - this.createIndicesAtStartTime = config.createIndicesAtStart(); - - writer = builder.build(); - writer.start(); - log.info( - "Started ElasticsearchSinkTask, will {} records with null values ('{}')", - config.behaviorOnNullValues().name(), - ElasticsearchSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG - ); - } catch (ConfigException e) { - throw new ConnectException( - "Couldn't start ElasticsearchSinkTask due to configuration error:", - e - ); + logTrace("Writing {} to Elasticsearch.", record); + + ensureIndexExists(convertTopicToIndexName(record.topic())); + checkMapping(record); + tryWriteRecord(record); } } @Override - public void open(Collection partitions) { - log.debug("Opening the task for topic partitions: {}", partitions); - if (createIndicesAtStartTime) { - Set topics = new HashSet<>(); - for (TopicPartition tp : partitions) { - topics.add(tp.topic()); - } - writer.createIndicesForTopics(topics); + public void flush(Map offsets) { + log.debug("Flushing data to Elasticsearch with the following offsets: {}", offsets); + try { + client.flush(); + } catch (IllegalStateException e) { + log.debug("Tried to flush data to Elasticsearch, but BulkProcessor is already closed.", e); } } @Override - public void put(Collection records) throws ConnectException { - log.debug("Putting {} records to Elasticsearch", records.size()); - writer.write(records); + public void stop() { + log.debug("Stopping Elasticsearch client."); + client.close(); } @Override - public void flush(Map offsets) { - if (writer != null) { - log.debug("Flushing data to Elasticsearch with the following offsets: {}", offsets); - writer.flush(); - } else { - log.debug("Could not flush data to Elasticsearch because ESWriter already closed."); + public String version() { + return Version.getVersion(); + } + + private void checkMapping(SinkRecord record) { + String index = convertTopicToIndexName(record.topic()); + if (!config.shouldIgnoreSchema(record.topic()) && !existingMappings.contains(index)) { + if (!client.hasMapping(index)) { + client.createMapping(index, record.valueSchema()); + } + log.debug("Caching mapping for index '{}' locally.", index); + existingMappings.add(index); } } - @Override - public void close(Collection partitions) { - log.debug("Closing the task for topic partitions: {}", partitions); + private String getServerVersion() { + ConfigCallbackHandler configCallbackHandler = new ConfigCallbackHandler(config); + RestHighLevelClient highLevelClient = new RestHighLevelClient( + RestClient + .builder( + config.connectionUrls() + .stream() + .map(HttpHost::create) + .collect(Collectors.toList()) + .toArray(new HttpHost[config.connectionUrls().size()]) + ) + .setHttpClientConfigCallback(configCallbackHandler) + ); + MainResponse response; + String esVersionNumber = "Unknown"; + try { + response = highLevelClient.info(RequestOptions.DEFAULT); + esVersionNumber = response.getVersion().getNumber(); + } catch (Exception e) { + // Same error messages as from validating the connection for IOException. + // Insufficient privileges to validate the version number if caught + // ElasticsearchStatusException. + log.warn("Failed to get ES server version", e); + } finally { + try { + highLevelClient.close(); + } catch (Exception e) { + log.warn("Failed to close high level client", e); + } + } + return esVersionNumber; } - @Override - public void stop() throws ConnectException { - log.info("Stopping ElasticsearchSinkTask"); - if (writer != null) { - writer.stop(); - writer = null; + /** + * Returns the converted index name from a given topic name. Elasticsearch accepts: + *
    + *
  • all lowercase
  • + *
  • less than 256 bytes
  • + *
  • does not start with - or _
  • + *
  • is not . or ..
  • + *
+ * (ref_.) + */ + private String convertTopicToIndexName(String topic) { + String index = topic.toLowerCase(); + if (index.length() > 255) { + index = index.substring(0, 255); + } + + if (index.startsWith("-") || index.startsWith("_")) { + index = index.substring(1); + } + + if (index.equals(".") || index.equals("..")) { + index = index.replace(".", "dot"); + log.warn("Elasticsearch cannot have indices named {}. Index will be named {}.", topic, index); + } + + if (!topic.equals(index)) { + log.trace("Topic '{}' was translated to index '{}'.", topic, index); + } + + return index; + } + + private void ensureIndexExists(String index) { + if (!indexCache.contains(index)) { + log.info("Creating index {}.", index); + client.createIndex(index); + indexCache.add(index); + } + } + + private void logTrace(String formatMsg, SinkRecord record) { + if (log.isTraceEnabled()) { + log.trace(formatMsg, recordString(record)); } - if (client != null) { - client.close(); + } + + private void reportBadRecord(SinkRecord record, Throwable error) { + if (reporter != null) { + reporter.report(record, error); } } + + private boolean shouldSkipRecord(SinkRecord record) { + return record.value() == null && config.behaviorOnNullValues() == BehaviorOnNullValues.IGNORE; + } + + private void tryWriteRecord(SinkRecord sinkRecord) { + DocWriteRequest record = null; + try { + record = converter.convertRecord(sinkRecord, convertTopicToIndexName(sinkRecord.topic())); + } catch (DataException convertException) { + reportBadRecord(sinkRecord, convertException); + + if (config.dropInvalidMessage()) { + log.error("Can't convert {}.", recordString(sinkRecord), convertException); + } else { + throw convertException; + } + } + + if (record != null) { + log.trace("Adding {} to bulk processor.", recordString(sinkRecord)); + client.index(sinkRecord, record); + } + } + + private static String recordString(SinkRecord record) { + return String.format( + "record from topic=%s partition=%s offset=%s", + record.topic(), + record.kafkaPartition(), + record.kafkaOffset() + ); + } } diff --git a/src/main/java/io/confluent/connect/elasticsearch/ElasticsearchWriter.java b/src/main/java/io/confluent/connect/elasticsearch/ElasticsearchWriter.java deleted file mode 100644 index 82e41b078..000000000 --- a/src/main/java/io/confluent/connect/elasticsearch/ElasticsearchWriter.java +++ /dev/null @@ -1,385 +0,0 @@ -/* - * 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.connect.elasticsearch; - -import io.confluent.connect.elasticsearch.bulk.BulkProcessor; -import org.apache.kafka.common.utils.SystemTime; -import org.apache.kafka.connect.errors.ConnectException; -import org.apache.kafka.connect.sink.ErrantRecordReporter; -import org.apache.kafka.connect.sink.SinkRecord; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.util.Collections; -import java.util.Collection; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Map; -import java.util.Objects; -import java.util.Set; - -import static io.confluent.connect.elasticsearch.DataConverter.BehaviorOnNullValues; -import static io.confluent.connect.elasticsearch.bulk.BulkProcessor.BehaviorOnMalformedDoc; - -public class ElasticsearchWriter { - private static final Logger log = LoggerFactory.getLogger(ElasticsearchWriter.class); - - private final ElasticsearchClient client; - private final String type; - private final boolean ignoreKey; - private final Set ignoreKeyTopics; - private final boolean ignoreSchema; - private final Set ignoreSchemaTopics; - @Deprecated - private final Map topicToIndexMap; - private final long flushTimeoutMs; - private final BulkProcessor bulkProcessor; - private final boolean dropInvalidMessage; - private final BehaviorOnNullValues behaviorOnNullValues; - private final DataConverter converter; - - private final Set existingMappings; - - ElasticsearchWriter( - ElasticsearchClient client, - String type, - boolean useCompactMapEntries, - boolean ignoreKey, - Set ignoreKeyTopics, - boolean ignoreSchema, - Set ignoreSchemaTopics, - Map topicToIndexMap, - long flushTimeoutMs, - int maxBufferedRecords, - int maxInFlightRequests, - int batchSize, - long lingerMs, - int maxRetries, - long retryBackoffMs, - boolean dropInvalidMessage, - BehaviorOnNullValues behaviorOnNullValues, - BehaviorOnMalformedDoc behaviorOnMalformedDoc, - ErrantRecordReporter reporter - ) { - this.client = client; - this.type = type; - this.ignoreKey = ignoreKey; - this.ignoreKeyTopics = ignoreKeyTopics; - this.ignoreSchema = ignoreSchema; - this.ignoreSchemaTopics = ignoreSchemaTopics; - this.topicToIndexMap = topicToIndexMap; - this.flushTimeoutMs = flushTimeoutMs; - this.dropInvalidMessage = dropInvalidMessage; - this.behaviorOnNullValues = behaviorOnNullValues; - this.converter = new DataConverter(useCompactMapEntries, behaviorOnNullValues); - - bulkProcessor = new BulkProcessor<>( - new SystemTime(), - new BulkIndexingClient(client), - maxBufferedRecords, - maxInFlightRequests, - batchSize, - lingerMs, - maxRetries, - retryBackoffMs, - behaviorOnMalformedDoc, - reporter - ); - - existingMappings = new HashSet<>(); - } - - public static class Builder { - private final ElasticsearchClient client; - private String type; - private boolean useCompactMapEntries = true; - private boolean ignoreKey = false; - private Set ignoreKeyTopics = Collections.emptySet(); - private boolean ignoreSchema = false; - private Set ignoreSchemaTopics = Collections.emptySet(); - private Map topicToIndexMap = new HashMap<>(); - private long flushTimeoutMs; - private int maxBufferedRecords; - private int maxInFlightRequests; - private int batchSize; - private long lingerMs; - private int maxRetry; - private long retryBackoffMs; - private boolean dropInvalidMessage; - private BehaviorOnNullValues behaviorOnNullValues = BehaviorOnNullValues.DEFAULT; - private BehaviorOnMalformedDoc behaviorOnMalformedDoc; - private ErrantRecordReporter reporter; - - public Builder(ElasticsearchClient client) { - this.client = client; - } - - public Builder setType(String type) { - this.type = type; - return this; - } - - public Builder setIgnoreKey(boolean ignoreKey, Set ignoreKeyTopics) { - this.ignoreKey = ignoreKey; - this.ignoreKeyTopics = ignoreKeyTopics; - return this; - } - - public Builder setIgnoreSchema(boolean ignoreSchema, Set ignoreSchemaTopics) { - this.ignoreSchema = ignoreSchema; - this.ignoreSchemaTopics = ignoreSchemaTopics; - return this; - } - - public Builder setCompactMapEntries(boolean useCompactMapEntries) { - this.useCompactMapEntries = useCompactMapEntries; - return this; - } - - public Builder setTopicToIndexMap(Map topicToIndexMap) { - this.topicToIndexMap = topicToIndexMap; - return this; - } - - public Builder setFlushTimoutMs(long flushTimeoutMs) { - this.flushTimeoutMs = flushTimeoutMs; - return this; - } - - public Builder setMaxBufferedRecords(int maxBufferedRecords) { - this.maxBufferedRecords = maxBufferedRecords; - return this; - } - - public Builder setMaxInFlightRequests(int maxInFlightRequests) { - this.maxInFlightRequests = maxInFlightRequests; - return this; - } - - public Builder setBatchSize(int batchSize) { - this.batchSize = batchSize; - return this; - } - - public Builder setLingerMs(long lingerMs) { - this.lingerMs = lingerMs; - return this; - } - - public Builder setMaxRetry(int maxRetry) { - this.maxRetry = maxRetry; - return this; - } - - public Builder setRetryBackoffMs(long retryBackoffMs) { - this.retryBackoffMs = retryBackoffMs; - return this; - } - - public Builder setDropInvalidMessage(boolean dropInvalidMessage) { - this.dropInvalidMessage = dropInvalidMessage; - return this; - } - - /** - * Change the behavior that the resulting {@link ElasticsearchWriter} will have when it - * encounters records with null values. - * @param behaviorOnNullValues Cannot be null. If in doubt, {@link BehaviorOnNullValues#DEFAULT} - * can be used. - */ - public Builder setBehaviorOnNullValues(BehaviorOnNullValues behaviorOnNullValues) { - this.behaviorOnNullValues = - Objects.requireNonNull(behaviorOnNullValues, "behaviorOnNullValues cannot be null"); - return this; - } - - public Builder setBehaviorOnMalformedDoc(BehaviorOnMalformedDoc behaviorOnMalformedDoc) { - this.behaviorOnMalformedDoc = behaviorOnMalformedDoc; - return this; - } - - public Builder setErrantRecordReporter(ErrantRecordReporter reporter) { - this.reporter = reporter; - return this; - } - - public ElasticsearchWriter build() { - return new ElasticsearchWriter( - client, - type, - useCompactMapEntries, - ignoreKey, - ignoreKeyTopics, - ignoreSchema, - ignoreSchemaTopics, - topicToIndexMap, - flushTimeoutMs, - maxBufferedRecords, - maxInFlightRequests, - batchSize, - lingerMs, - maxRetry, - retryBackoffMs, - dropInvalidMessage, - behaviorOnNullValues, - behaviorOnMalformedDoc, - reporter - ); - } - } - - public void write(Collection records) { - for (SinkRecord sinkRecord : records) { - // Preemptively skip records with null values if they're going to be ignored anyways - if (ignoreRecord(sinkRecord)) { - log.trace( - "Ignoring sink record with null value for topic/partition/offset {}/{}/{}", - sinkRecord.topic(), - sinkRecord.kafkaPartition(), - sinkRecord.kafkaOffset() - ); - continue; - } - log.trace("Writing record to Elasticsearch: topic/partition/offset {}/{}/{}", - sinkRecord.topic(), - sinkRecord.kafkaPartition(), - sinkRecord.kafkaOffset() - ); - - final String index = convertTopicToIndexName(sinkRecord.topic()); - final boolean ignoreKey = ignoreKeyTopics.contains(sinkRecord.topic()) || this.ignoreKey; - final boolean ignoreSchema = - ignoreSchemaTopics.contains(sinkRecord.topic()) || this.ignoreSchema; - - client.createIndices(Collections.singleton(index)); - - if (!ignoreSchema && !existingMappings.contains(index)) { - try { - if (Mapping.getMapping(client, index, type) == null) { - Mapping.createMapping(client, index, type, sinkRecord.valueSchema()); - } - } catch (IOException e) { - // FIXME: concurrent tasks could attempt to create the mapping and one of the requests may - // fail - throw new ConnectException("Failed to initialize mapping for index: " + index, e); - } - log.debug("Locally caching mapping for index '{}'", index); - existingMappings.add(index); - } - - tryWriteRecord(sinkRecord, index, ignoreKey, ignoreSchema); - } - } - - private boolean ignoreRecord(SinkRecord record) { - return record.value() == null && behaviorOnNullValues == BehaviorOnNullValues.IGNORE; - } - - private void tryWriteRecord( - SinkRecord sinkRecord, - String index, - boolean ignoreKey, - boolean ignoreSchema) { - IndexableRecord record = null; - try { - record = converter.convertRecord( - sinkRecord, - index, - type, - ignoreKey, - ignoreSchema); - } catch (ConnectException convertException) { - if (dropInvalidMessage) { - log.error( - "Can't convert record from topic/partition/offset {}/{}/{}. " - + "Error message: {}", - sinkRecord.topic(), - sinkRecord.kafkaPartition(), - sinkRecord.kafkaOffset(), - convertException.getMessage() - ); - } else { - throw convertException; - } - } - if (record != null) { - log.trace( - "Adding record from topic/partition/offset {}/{}/{} to bulk processor", - sinkRecord.topic(), - sinkRecord.kafkaPartition(), - sinkRecord.kafkaOffset() - ); - bulkProcessor.add(record, sinkRecord, flushTimeoutMs); - } - } - - /** - * Return the expected index name for a given topic, using the configured mapping or the topic - * name. Elasticsearch accepts only lowercase index names - * (ref_. - */ - private String convertTopicToIndexName(String topic) { - final String indexOverride = topicToIndexMap.get(topic); - String index = indexOverride != null ? indexOverride : topic.toLowerCase(); - log.trace("Topic '{}' was translated as index '{}'", topic, index); - return index; - } - - public void flush() { - bulkProcessor.flush(flushTimeoutMs); - } - - public void start() { - bulkProcessor.start(); - } - - public void stop() { - try { - log.debug( - "Flushing records, waiting up to {}ms ('{}')", - flushTimeoutMs, - ElasticsearchSinkConnectorConfig.FLUSH_TIMEOUT_MS_CONFIG - ); - bulkProcessor.flush(flushTimeoutMs); - } catch (Exception e) { - log.warn("Failed to flush during stop", e); - } - log.debug("Stopping Elastisearch writer"); - bulkProcessor.stop(); - log.debug( - "Waiting for bulk processor to stop, up to {}ms ('{}')", - flushTimeoutMs, - ElasticsearchSinkConnectorConfig.FLUSH_TIMEOUT_MS_CONFIG - ); - bulkProcessor.awaitStop(flushTimeoutMs); - log.debug("Stopped Elastisearch writer"); - } - - public void createIndicesForTopics(Set assignedTopics) { - Objects.requireNonNull(assignedTopics); - client.createIndices(indicesForTopics(assignedTopics)); - } - - private Set indicesForTopics(Set assignedTopics) { - final Set indices = new HashSet<>(); - for (String topic : assignedTopics) { - indices.add(convertTopicToIndexName(topic)); - } - return indices; - } - -} diff --git a/src/main/java/io/confluent/connect/elasticsearch/EnumRecommender.java b/src/main/java/io/confluent/connect/elasticsearch/EnumRecommender.java new file mode 100644 index 000000000..5b8a6982b --- /dev/null +++ b/src/main/java/io/confluent/connect/elasticsearch/EnumRecommender.java @@ -0,0 +1,69 @@ +/* + * 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.connect.elasticsearch; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.config.ConfigException; + +class EnumRecommender> implements ConfigDef.Validator, ConfigDef.Recommender { + + private final Set validValues; + private final Class enumClass; + + public EnumRecommender(Class enumClass) { + this.enumClass = enumClass; + Set validEnums = new LinkedHashSet<>(); + for (Object o : enumClass.getEnumConstants()) { + String key = o.toString().toLowerCase(); + validEnums.add(key); + } + + this.validValues = Collections.unmodifiableSet(validEnums); + } + + @Override + public void ensureValid(String key, Object value) { + if (value == null) { + return; + } + String enumValue = value.toString().toLowerCase(); + if (value != null && !validValues.contains(enumValue)) { + throw new ConfigException(key, value, "Value must be one of: " + this); + } + } + + @Override + public String toString() { + return validValues.toString(); + } + + @Override + public List validValues(String name, Map connectorConfigs) { + return Collections.unmodifiableList(new ArrayList<>(validValues)); + } + + @Override + public boolean visible(String name, Map connectorConfigs) { + return true; + } +} diff --git a/src/main/java/io/confluent/connect/elasticsearch/IndexableRecord.java b/src/main/java/io/confluent/connect/elasticsearch/IndexableRecord.java deleted file mode 100644 index 2f1d16c57..000000000 --- a/src/main/java/io/confluent/connect/elasticsearch/IndexableRecord.java +++ /dev/null @@ -1,50 +0,0 @@ -/* - * 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.connect.elasticsearch; - -import java.util.Objects; - -public class IndexableRecord { - - public final Key key; - public final String payload; - public final Long version; - - public IndexableRecord(Key key, String payload, Long version) { - this.key = key; - this.version = version; - this.payload = payload; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - IndexableRecord that = (IndexableRecord) o; - return Objects.equals(key, that.key) - && Objects.equals(payload, that.payload) - && Objects.equals(version, that.version); - } - - @Override - public int hashCode() { - return Objects.hash(key, version, payload); - } -} diff --git a/src/main/java/io/confluent/connect/elasticsearch/Key.java b/src/main/java/io/confluent/connect/elasticsearch/Key.java deleted file mode 100644 index f59d572af..000000000 --- a/src/main/java/io/confluent/connect/elasticsearch/Key.java +++ /dev/null @@ -1,56 +0,0 @@ -/* - * 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.connect.elasticsearch; - -import java.util.Objects; - -public class Key { - - public final String index; - public final String type; - public final String id; - - public Key(String index, String type, String id) { - this.index = index; - this.type = type; - this.id = id; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - Key that = (Key) o; - return Objects.equals(index, that.index) - && Objects.equals(type, that.type) - && Objects.equals(id, that.id); - } - - @Override - public int hashCode() { - return Objects.hash(index, type, id); - } - - @Override - public String toString() { - return String.format("Key{%s/%s/%s}", index, type, id); - } - -} diff --git a/src/main/java/io/confluent/connect/elasticsearch/LogContext.java b/src/main/java/io/confluent/connect/elasticsearch/LogContext.java deleted file mode 100644 index f8bd365ef..000000000 --- a/src/main/java/io/confluent/connect/elasticsearch/LogContext.java +++ /dev/null @@ -1,56 +0,0 @@ -/* - * Copyright [2018 - 2018] Confluent Inc. - */ - -package io.confluent.connect.elasticsearch; - -import org.slf4j.MDC; - -/** - * Utility that works with Connect's MDC logging when it is enabled, so that the threads - * created by this connector also include the same connector task specific MDC context plus - * additional information that distinguishes those threads from each other and from the task thread. - */ -public class LogContext implements AutoCloseable { - - /** - * We can't reference Connect's constant, since this connector could be deployed to Connect - * runtimes that don't yet have it. - */ - private static final String CONNECTOR_CONTEXT = "connector.context"; - - private final String previousContext; - private final String currentContext; - - public LogContext() { - this(MDC.get(CONNECTOR_CONTEXT), null); - } - - protected LogContext(String currentContext, String suffix) { - this.previousContext = currentContext; - if (currentContext != null && suffix != null && !suffix.trim().isEmpty()) { - this.currentContext = currentContext + suffix.trim() + " "; - MDC.put(CONNECTOR_CONTEXT, this.currentContext); - } else { - this.currentContext = null; - } - } - - public LogContext create(String suffix) { - if (previousContext != null && suffix != null && !suffix.trim().isEmpty()) { - return new LogContext(previousContext, suffix); - } - return this; - } - - @Override - public void close() { - if (currentContext != null) { - if (previousContext != null) { - MDC.put(CONNECTOR_CONTEXT, previousContext); - } else { - MDC.remove(CONNECTOR_CONTEXT); - } - } - } -} diff --git a/src/main/java/io/confluent/connect/elasticsearch/Mapping.java b/src/main/java/io/confluent/connect/elasticsearch/Mapping.java index fc0ee87fb..cd14dfadd 100644 --- a/src/main/java/io/confluent/connect/elasticsearch/Mapping.java +++ b/src/main/java/io/confluent/connect/elasticsearch/Mapping.java @@ -15,10 +15,6 @@ package io.confluent.connect.elasticsearch; -import com.fasterxml.jackson.databind.JsonNode; -import com.fasterxml.jackson.databind.node.JsonNodeFactory; -import com.fasterxml.jackson.databind.node.ObjectNode; -import com.google.gson.JsonObject; import org.apache.kafka.connect.data.Date; import org.apache.kafka.connect.data.Decimal; import org.apache.kafka.connect.data.Field; @@ -27,94 +23,207 @@ import org.apache.kafka.connect.data.Timestamp; import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.errors.DataException; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentFactory; import java.io.IOException; -import java.nio.ByteBuffer; - -import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConstants.BINARY_TYPE; -import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConstants.BOOLEAN_TYPE; -import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConstants.BYTE_TYPE; -import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConstants.DOUBLE_TYPE; -import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConstants.FLOAT_TYPE; -import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConstants.INTEGER_TYPE; -import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConstants.KEYWORD_TYPE; -import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConstants.LONG_TYPE; -import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConstants.MAP_KEY; -import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConstants.MAP_VALUE; -import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConstants.SHORT_TYPE; -import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConstants.STRING_TYPE; -import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConstants.TEXT_TYPE; public class Mapping { + // Elasticsearch types + public static final String BOOLEAN_TYPE = "boolean"; + public static final String BYTE_TYPE = "byte"; + public static final String BINARY_TYPE = "binary"; + public static final String SHORT_TYPE = "short"; + public static final String INTEGER_TYPE = "integer"; + public static final String LONG_TYPE = "long"; + public static final String FLOAT_TYPE = "float"; + public static final String DOUBLE_TYPE = "double"; + public static final String STRING_TYPE = "string"; + public static final String TEXT_TYPE = "text"; + public static final String KEYWORD_TYPE = "keyword"; + public static final String DATE_TYPE = "date"; + + // Elasticsearch mapping fields + private static final String DEFAULT_VALUE_FIELD = "null_value"; + private static final String FIELDS_FIELD = "fields"; + private static final String IGNORE_ABOVE_FIELD = "ignore_above"; + public static final String KEY_FIELD = "key"; + private static final String KEYWORD_FIELD = "keyword"; + private static final String PROPERTIES_FIELD = "properties"; + private static final String TYPE_FIELD = "type"; + public static final String VALUE_FIELD = "value"; + /** - * Create an explicit mapping. + * Build mapping from the provided schema. * - * @param client The client to connect to Elasticsearch. - * @param index The index to write to Elasticsearch. - * @param type The type to create mapping for. - * @param schema The schema used to infer mapping. - * @throws IOException from underlying JestClient + * @param schema The schema used to build the mapping. + * @return the schema as a JSON mapping */ - public static void createMapping( - ElasticsearchClient client, - String index, - String type, - Schema schema - ) - throws IOException { - client.createMapping(index, type, schema); + public static XContentBuilder buildMapping(Schema schema) { + try { + XContentBuilder builder = XContentFactory.jsonBuilder(); + builder.startObject(); + { + buildMapping(schema, builder); + } + builder.endObject(); + return builder; + } catch (IOException e) { + throw new ConnectException("Failed to build mapping for schema " + schema, e); + } } - /** - * Get the JSON mapping for given index and type. Returns {@code null} if it does not exist. - */ - public static JsonObject getMapping(ElasticsearchClient client, String index, String type) + private static XContentBuilder buildMapping(Schema schema, XContentBuilder builder) throws IOException { - return client.getMapping(index, type); - } - /** - * Infer mapping from the provided schema. - * - * @param schema The schema used to infer mapping. - */ - public static JsonNode inferMapping(ElasticsearchClient client, Schema schema) { if (schema == null) { throw new DataException("Cannot infer mapping without schema."); } // Handle logical types - JsonNode logicalConversion = inferLogicalMapping(schema); + XContentBuilder logicalConversion = inferLogicalMapping(builder, schema); if (logicalConversion != null) { return logicalConversion; } Schema.Type schemaType = schema.type(); - ObjectNode properties = JsonNodeFactory.instance.objectNode(); - ObjectNode fields = JsonNodeFactory.instance.objectNode(); - switch (schemaType) { + switch (schema.type()) { case ARRAY: - return inferMapping(client, schema.valueSchema()); + return buildMapping(schema.valueSchema(), builder); + case MAP: - properties.set("properties", fields); - fields.set(MAP_KEY, inferMapping(client, schema.keySchema())); - fields.set(MAP_VALUE, inferMapping(client, schema.valueSchema())); - return properties; + return buildMap(schema, builder); + case STRUCT: - properties.set("properties", fields); - for (Field field : schema.fields()) { - fields.set(field.name(), inferMapping(client, field.schema())); + return buildStruct(schema, builder); + + default: + return inferPrimitive(builder, getElasticsearchType(schemaType), schema.defaultValue()); + } + } + + private static void addTextMapping(XContentBuilder builder) throws IOException { + // Add additional mapping for indexing, per https://www.elastic.co/blog/strings-are-dead-long-live-strings + builder.startObject(FIELDS_FIELD); + { + builder.startObject(KEYWORD_FIELD); + { + builder.field(TYPE_FIELD, KEYWORD_TYPE); + builder.field(IGNORE_ABOVE_FIELD, 256); + } + builder.endObject(); + } + builder.endObject(); + } + + private static XContentBuilder buildMap(Schema schema, XContentBuilder builder) + throws IOException { + + builder.startObject(PROPERTIES_FIELD); + { + builder.startObject(KEY_FIELD); + { + buildMapping(schema.keySchema(), builder); + } + builder.endObject(); + builder.startObject(VALUE_FIELD); + { + buildMapping(schema.valueSchema(), builder); + } + builder.endObject(); + } + return builder.endObject(); + } + + private static XContentBuilder buildStruct(Schema schema, XContentBuilder builder) + throws IOException { + + builder.startObject(PROPERTIES_FIELD); + { + for (Field field : schema.fields()) { + builder.startObject(field.name()); + { + buildMapping(field.schema(), builder); } - return properties; + builder.endObject(); + } + } + return builder.endObject(); + } + + private static XContentBuilder inferPrimitive( + XContentBuilder builder, + String type, + Object defaultValue + ) throws IOException { + + if (type == null) { + throw new DataException(String.format("Invalid primitive type %s.", type)); + } + + builder.field(TYPE_FIELD, type); + if (type.equals(TEXT_TYPE)) { + addTextMapping(builder); + } + + if (defaultValue == null) { + return builder; + } + + switch (type) { + case BYTE_TYPE: + return builder.field(DEFAULT_VALUE_FIELD, (byte) defaultValue); + case SHORT_TYPE: + return builder.field(DEFAULT_VALUE_FIELD, (short) defaultValue); + case INTEGER_TYPE: + return builder.field(DEFAULT_VALUE_FIELD, (int) defaultValue); + case LONG_TYPE: + return builder.field(DEFAULT_VALUE_FIELD, (long) defaultValue); + case FLOAT_TYPE: + return builder.field(DEFAULT_VALUE_FIELD, (float) defaultValue); + case DOUBLE_TYPE: + return builder.field(DEFAULT_VALUE_FIELD, (double) defaultValue); + case BOOLEAN_TYPE: + return builder.field(DEFAULT_VALUE_FIELD, (boolean) defaultValue); + case DATE_TYPE: + return builder.field(DEFAULT_VALUE_FIELD, ((java.util.Date) defaultValue).getTime()); + /* + * IGNORE default values for text and binary types as this is not supported by ES side. + * see https://www.elastic.co/guide/en/elasticsearch/reference/current/text.html and + * https://www.elastic.co/guide/en/elasticsearch/reference/current/binary.html for details. + */ + case STRING_TYPE: + case TEXT_TYPE: + case BINARY_TYPE: + return builder; default: - String esType = getElasticsearchType(client, schemaType); - return inferPrimitive(esType, schema.defaultValue()); + throw new DataException("Invalid primitive type " + type + "."); + } + } + + private static XContentBuilder inferLogicalMapping(XContentBuilder builder, Schema schema) + throws IOException { + + if (schema.name() == null) { + return null; + } + + switch (schema.name()) { + case Date.LOGICAL_NAME: + case Time.LOGICAL_NAME: + case Timestamp.LOGICAL_NAME: + return inferPrimitive(builder, DATE_TYPE, schema.defaultValue()); + case Decimal.LOGICAL_NAME: + return inferPrimitive(builder, DOUBLE_TYPE, schema.defaultValue()); + default: + // User-defined type or unknown built-in + return null; } } // visible for testing - protected static String getElasticsearchType(ElasticsearchClient client, Schema.Type schemaType) { + protected static String getElasticsearchType(Schema.Type schemaType) { switch (schemaType) { case BOOLEAN: return BOOLEAN_TYPE; @@ -131,119 +240,11 @@ protected static String getElasticsearchType(ElasticsearchClient client, Schema. case FLOAT64: return DOUBLE_TYPE; case STRING: - switch (client.getVersion()) { - case ES_V1: - case ES_V2: - return STRING_TYPE; - case ES_V5: - case ES_V6: - default: - return TEXT_TYPE; - } + return TEXT_TYPE; case BYTES: return BINARY_TYPE; default: return null; } } - - private static JsonNode inferLogicalMapping(Schema schema) { - String schemaName = schema.name(); - Object defaultValue = schema.defaultValue(); - if (schemaName == null) { - return null; - } - - switch (schemaName) { - case Date.LOGICAL_NAME: - case Time.LOGICAL_NAME: - case Timestamp.LOGICAL_NAME: - return inferPrimitive(ElasticsearchSinkConnectorConstants.DATE_TYPE, defaultValue); - case Decimal.LOGICAL_NAME: - return inferPrimitive(ElasticsearchSinkConnectorConstants.DOUBLE_TYPE, defaultValue); - default: - // User-defined type or unknown built-in - return null; - } - } - - private static JsonNode inferPrimitive(String type, Object defaultValue) { - if (type == null) { - throw new ConnectException("Invalid primitive type."); - } - - ObjectNode obj = JsonNodeFactory.instance.objectNode(); - obj.set("type", JsonNodeFactory.instance.textNode(type)); - if (type.equals(TEXT_TYPE)) { - addTextMapping(obj); - } - JsonNode defaultValueNode = null; - if (defaultValue != null) { - switch (type) { - case ElasticsearchSinkConnectorConstants.BYTE_TYPE: - defaultValueNode = JsonNodeFactory.instance.numberNode((byte) defaultValue); - break; - case ElasticsearchSinkConnectorConstants.SHORT_TYPE: - defaultValueNode = JsonNodeFactory.instance.numberNode((short) defaultValue); - break; - case ElasticsearchSinkConnectorConstants.INTEGER_TYPE: - defaultValueNode = JsonNodeFactory.instance.numberNode((int) defaultValue); - break; - case ElasticsearchSinkConnectorConstants.LONG_TYPE: - defaultValueNode = JsonNodeFactory.instance.numberNode((long) defaultValue); - break; - case ElasticsearchSinkConnectorConstants.FLOAT_TYPE: - defaultValueNode = JsonNodeFactory.instance.numberNode((float) defaultValue); - break; - case ElasticsearchSinkConnectorConstants.DOUBLE_TYPE: - defaultValueNode = JsonNodeFactory.instance.numberNode((double) defaultValue); - break; - case ElasticsearchSinkConnectorConstants.STRING_TYPE: - case ElasticsearchSinkConnectorConstants.TEXT_TYPE: - case ElasticsearchSinkConnectorConstants.BINARY_TYPE: - // IGNORE default values for text and binary types as this is not supported by ES side. - // see https://www.elastic.co/guide/en/elasticsearch/reference/current/text.html - // https://www.elastic.co/guide/en/elasticsearch/reference/current/binary.html - // for more details. - //defaultValueNode = null; - break; - case ElasticsearchSinkConnectorConstants.BOOLEAN_TYPE: - defaultValueNode = JsonNodeFactory.instance.booleanNode((boolean) defaultValue); - break; - case ElasticsearchSinkConnectorConstants.DATE_TYPE: - long value = ((java.util.Date) defaultValue).getTime(); - defaultValueNode = JsonNodeFactory.instance.numberNode(value); - break; - default: - throw new DataException("Invalid primitive type."); - } - } - if (defaultValueNode != null) { - obj.set("null_value", defaultValueNode); - } - return obj; - } - - private static void addTextMapping(ObjectNode obj) { - // Add additional mapping for indexing, per https://www.elastic.co/blog/strings-are-dead-long-live-strings - ObjectNode keyword = JsonNodeFactory.instance.objectNode(); - keyword.set("type", JsonNodeFactory.instance.textNode(KEYWORD_TYPE)); - keyword.set("ignore_above", JsonNodeFactory.instance.numberNode(256)); - ObjectNode fields = JsonNodeFactory.instance.objectNode(); - fields.set("keyword", keyword); - obj.set("fields", fields); - } - - private static byte[] bytes(Object value) { - final byte[] bytes; - if (value instanceof ByteBuffer) { - final ByteBuffer buffer = ((ByteBuffer) value).slice(); - bytes = new byte[buffer.remaining()]; - buffer.get(bytes); - } else { - bytes = (byte[]) value; - } - return bytes; - } - } diff --git a/src/main/java/io/confluent/connect/elasticsearch/RetryUtil.java b/src/main/java/io/confluent/connect/elasticsearch/RetryUtil.java index 8d0212770..8d968e99c 100644 --- a/src/main/java/io/confluent/connect/elasticsearch/RetryUtil.java +++ b/src/main/java/io/confluent/connect/elasticsearch/RetryUtil.java @@ -15,9 +15,15 @@ package io.confluent.connect.elasticsearch; +import java.util.concurrent.Callable; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.connect.errors.ConnectException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + /** * Utility to compute the retry times for a given attempt, using exponential backoff. * @@ -31,6 +37,8 @@ */ public class RetryUtil { + private static final Logger log = LoggerFactory.getLogger(RetryUtil.class); + /** * An arbitrary absolute maximum practical retry time. */ @@ -86,5 +94,84 @@ public static long computeRetryWaitTimeInMillis(int retryAttempts, return result < 0L ? MAX_RETRY_TIME_MS : Math.min(MAX_RETRY_TIME_MS, result); } + /** + * Call the supplied function up to the {@code maxTotalAttempts}. + * + *

The description of the function should be a succinct, human-readable present tense phrase + * that summarizes the function, such as "read tables" or "connect to database" or + * "make remote request". This description will be used within exception and log messages. + * + * @param description present tense description of the action, used to create the error + * message; may not be null + * @param function the function to call; may not be null + * @param maxTotalAttempts maximum number of total attempts, including the first call + * @param initialBackoff the initial backoff in ms before retrying + * @param the return type of the function to retry + * @return the function's return value + * @throws ConnectException if the function failed after retries + */ + public static T callWithRetries( + String description, + Callable function, + int maxTotalAttempts, + long initialBackoff + ) { + return callWithRetries(description, function, maxTotalAttempts, initialBackoff, Time.SYSTEM); + } + /** + * Call the supplied function up to the {@code maxTotalAttempts}. + * + *

The description of the function should be a succinct, human-readable present tense phrase + * that summarizes the function, such as "read tables" or "connect to database" or + * "make remote request". This description will be used within exception and log messages. + * + * @param description present tense description of the action, used to create the error + * message; may not be null + * @param function the function to call; may not be null + * @param maxTotalAttempts maximum number of attempts + * @param initialBackoff the initial backoff in ms before retrying + * @param clock the clock to use for waiting + * @param the return type of the function to retry + * @return the function's return value + * @throws ConnectException if the function failed after retries + */ + protected static T callWithRetries( + String description, + Callable function, + int maxTotalAttempts, + long initialBackoff, + Time clock + ) { + assert description != null; + assert function != null; + int attempt = 0; + while (true) { + ++attempt; + try { + log.trace( + "Try {} (attempt {} of {})", + description, + attempt, + maxTotalAttempts + ); + T call = function.call(); + return call; + } catch (Exception e) { + if (attempt >= maxTotalAttempts) { + String msg = String.format("Failed to %s due to '%s' after %d attempt(s)", + description, e, attempt); + log.error(msg, e); + throw new ConnectException(msg, e); + } + + // Otherwise it is retriable and we should retry + long backoff = computeRandomRetryWaitTimeInMillis(attempt, initialBackoff); + + log.warn("Failed to {} due to {}. Retrying attempt ({}/{}) after backoff of {} ms", + description, e.getCause(), attempt, maxTotalAttempts, backoff); + clock.sleep(backoff); + } + } + } } diff --git a/src/main/java/io/confluent/connect/elasticsearch/SecurityProtocol.java b/src/main/java/io/confluent/connect/elasticsearch/SecurityProtocol.java deleted file mode 100644 index 359ed4dae..000000000 --- a/src/main/java/io/confluent/connect/elasticsearch/SecurityProtocol.java +++ /dev/null @@ -1,36 +0,0 @@ -/* - * Copyright 2019 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.connect.elasticsearch; - -public enum SecurityProtocol { - - /** - * Un-authenticated, non-encrypted channel - */ - PLAINTEXT(0, "PLAINTEXT"), - /** - * SSL channel - */ - SSL(1, "SSL"); - - private final int id; - private final String name; - - SecurityProtocol(int id, String name) { - this.id = id; - this.name = name; - } -} diff --git a/src/main/java/io/confluent/connect/elasticsearch/Validator.java b/src/main/java/io/confluent/connect/elasticsearch/Validator.java index 14747aa56..671861611 100644 --- a/src/main/java/io/confluent/connect/elasticsearch/Validator.java +++ b/src/main/java/io/confluent/connect/elasticsearch/Validator.java @@ -1,60 +1,83 @@ /* * Copyright 2020 Confluent Inc. * - * Licensed 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 + * 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.apache.org/licenses/LICENSE-2.0 + * 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 OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. - **/ + * 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.connect.elasticsearch; +import org.apache.http.HttpHost; +import org.apache.kafka.common.config.Config; +import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.common.config.ConfigValue; +import org.apache.kafka.common.config.SslConfigs; +import org.elasticsearch.ElasticsearchStatusException; +import org.elasticsearch.client.RequestOptions; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.RestHighLevelClient; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.function.Function; +import java.util.stream.Collectors; + +import io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.SecurityProtocol; + import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.BATCH_SIZE_CONFIG; import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.CONNECTION_PASSWORD_CONFIG; +import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.CONNECTION_URL_CONFIG; import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.CONNECTION_USERNAME_CONFIG; import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.FLUSH_TIMEOUT_MS_CONFIG; import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.IGNORE_KEY_CONFIG; import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.IGNORE_KEY_TOPICS_CONFIG; import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.IGNORE_SCHEMA_CONFIG; import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.IGNORE_SCHEMA_TOPICS_CONFIG; +import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.KERBEROS_KEYTAB_PATH_CONFIG; +import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.KERBEROS_PRINCIPAL_CONFIG; import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.LINGER_MS_CONFIG; import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.MAX_BUFFERED_RECORDS_CONFIG; import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.MAX_IN_FLIGHT_REQUESTS_CONFIG; import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.PROXY_HOST_CONFIG; import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.PROXY_PASSWORD_CONFIG; +import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.PROXY_PORT_CONFIG; import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.PROXY_USERNAME_CONFIG; import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.SECURITY_PROTOCOL_CONFIG; import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.SSL_CONFIG_PREFIX; -import java.util.List; -import java.util.Map; -import java.util.function.Function; -import java.util.stream.Collectors; -import org.apache.kafka.common.config.Config; -import org.apache.kafka.common.config.ConfigException; -import org.apache.kafka.common.config.ConfigValue; -import org.apache.kafka.common.config.SslConfigs; - public class Validator { + private static final Logger log = LoggerFactory.getLogger(Validator.class); + private ElasticsearchSinkConnectorConfig config; private Map values; private List validations; + private ClientFactory clientFactory; public Validator(Map props) { + this(props, null); + } + + // Exposed for testing + protected Validator(Map props, ClientFactory clientFactory) { try { this.config = new ElasticsearchSinkConnectorConfig(props); } catch (ConfigException e) { // some configs are invalid } + this.clientFactory = clientFactory == null ? this::createClient : clientFactory; validations = ElasticsearchSinkConnectorConfig.CONFIG.validate(props); values = validations.stream().collect(Collectors.toMap(ConfigValue::name, Function.identity())); } @@ -65,17 +88,26 @@ public Config validate() { return new Config(validations); } - validateCredentials(); - validateIgnoreConfigs(); - validateLingerMs(); - validateMaxBufferedRecords(); - validateProxy(); - validateSsl(); + try (RestHighLevelClient client = clientFactory.client()) { + validateCredentials(); + validateIgnoreConfigs(); + validateKerberos(); + validateLingerMs(); + validateMaxBufferedRecords(); + validateProxy(); + validateSsl(); + + if (!hasErrors()) { + // no point if previous configs are invalid + validateConnection(client); + } + } catch (IOException e) { + log.warn("Closing the client failed.", e); + } return new Config(validations); } - private void validateCredentials() { boolean onlyOneSet = config.username() != null ^ config.password() != null; if (onlyOneSet) { @@ -105,6 +137,50 @@ private void validateIgnoreConfigs() { } } + private void validateKerberos() { + boolean onlyOneSet = config.kerberosUserPrincipal() != null ^ config.keytabPath() != null; + if (onlyOneSet) { + String errorMessage = String.format( + "Either both or neither '%s' and '%s' must be set.", + KERBEROS_PRINCIPAL_CONFIG, + KERBEROS_KEYTAB_PATH_CONFIG + ); + addErrorMessage(KERBEROS_PRINCIPAL_CONFIG, errorMessage); + addErrorMessage(KERBEROS_KEYTAB_PATH_CONFIG, errorMessage); + } + + if (config.isKerberosEnabled()) { + // currently do not support Kerberos with regular auth + if (config.isAuthenticatedConnection()) { + String errorMessage = String.format( + "Either only Kerberos (%s, %s) or connection credentials (%s, %s) must be set.", + KERBEROS_PRINCIPAL_CONFIG, + KERBEROS_KEYTAB_PATH_CONFIG, + CONNECTION_USERNAME_CONFIG, + CONNECTION_PASSWORD_CONFIG + ); + addErrorMessage(KERBEROS_PRINCIPAL_CONFIG, errorMessage); + addErrorMessage(KERBEROS_KEYTAB_PATH_CONFIG, errorMessage); + addErrorMessage(CONNECTION_USERNAME_CONFIG, errorMessage); + addErrorMessage(CONNECTION_PASSWORD_CONFIG, errorMessage); + } + + // currently do not support Kerberos with proxy + if (config.isBasicProxyConfigured()) { + String errorMessage = String.format( + "Kerberos (%s, %s) is not supported with proxy settings (%s).", + KERBEROS_PRINCIPAL_CONFIG, + KERBEROS_KEYTAB_PATH_CONFIG, + PROXY_HOST_CONFIG + ); + addErrorMessage(KERBEROS_PRINCIPAL_CONFIG, errorMessage); + addErrorMessage(KERBEROS_KEYTAB_PATH_CONFIG, errorMessage); + addErrorMessage(PROXY_HOST_CONFIG, errorMessage); + } + } + + } + private void validateLingerMs() { if (config.lingerMs() > config.flushTimeoutMs()) { String errorMessage = String.format( @@ -164,7 +240,7 @@ private void validateProxy() { private void validateSsl() { Map sslConfigs = config.originalsWithPrefix(SSL_CONFIG_PREFIX); - if (!config.secured()) { + if (!config.isSslEnabled()) { if (!sslConfigs.isEmpty()) { String errorMessage = String.format( "'%s' must be set to '%s' to use SSL configs.", @@ -190,7 +266,108 @@ private void validateSsl() { } } + private void validateConnection(RestHighLevelClient client) { + boolean successful; + String exceptionMessage = ""; + try { + successful = client.ping(RequestOptions.DEFAULT); + } catch (ElasticsearchStatusException e) { + switch (e.status()) { + case FORBIDDEN: + // ES is up, but user is not authorized to ping server + successful = true; + break; + default: + successful = false; + exceptionMessage = String.format("Error message: %s", e.getMessage()); + } + } catch (Exception e) { + successful = false; + exceptionMessage = String.format("Error message: %s", e.getMessage()); + } + if (!successful) { + String errorMessage = String.format( + "Could not connect to Elasticsearch. %s", + exceptionMessage + ); + addErrorMessage(CONNECTION_URL_CONFIG, errorMessage); + + if (config.isAuthenticatedConnection()) { + errorMessage = String.format( + "Could not authenticate the user. Check the '%s' and '%s'. %s", + CONNECTION_USERNAME_CONFIG, + CONNECTION_PASSWORD_CONFIG, + exceptionMessage + ); + addErrorMessage(CONNECTION_USERNAME_CONFIG, errorMessage); + addErrorMessage(CONNECTION_PASSWORD_CONFIG, errorMessage); + } + + if (config.isSslEnabled()) { + errorMessage = String.format( + "Could not connect to Elasticsearch. Check your SSL settings.%s", + exceptionMessage + ); + + addErrorMessage(SECURITY_PROTOCOL_CONFIG, errorMessage); + } + + if (config.isKerberosEnabled()) { + errorMessage = String.format( + "Could not connect to Elasticsearch. Check your Kerberos settings. %s", + exceptionMessage + ); + + addErrorMessage(KERBEROS_PRINCIPAL_CONFIG, errorMessage); + addErrorMessage(KERBEROS_KEYTAB_PATH_CONFIG, errorMessage); + } + + if (config.isBasicProxyConfigured()) { + errorMessage = String.format( + "Could not connect to Elasticsearch. Check your proxy settings. %s", + exceptionMessage + ); + addErrorMessage(PROXY_HOST_CONFIG, errorMessage); + addErrorMessage(PROXY_PORT_CONFIG, errorMessage); + + if (config.isProxyWithAuthenticationConfigured()) { + addErrorMessage(PROXY_USERNAME_CONFIG, errorMessage); + addErrorMessage(PROXY_PASSWORD_CONFIG, errorMessage); + } + } + } + } + private void addErrorMessage(String property, String error) { values.get(property).addErrorMessage(error); } + + private RestHighLevelClient createClient() { + ConfigCallbackHandler configCallbackHandler = new ConfigCallbackHandler(config); + return new RestHighLevelClient( + RestClient + .builder( + config.connectionUrls() + .stream() + .map(HttpHost::create) + .collect(Collectors.toList()) + .toArray(new HttpHost[config.connectionUrls().size()]) + ) + .setHttpClientConfigCallback(configCallbackHandler) + ); + } + + private boolean hasErrors() { + for (ConfigValue config : validations) { + if (!config.errorMessages().isEmpty()) { + return true; + } + } + + return false; + } + + interface ClientFactory { + RestHighLevelClient client(); + } } diff --git a/src/main/java/io/confluent/connect/elasticsearch/bulk/BulkClient.java b/src/main/java/io/confluent/connect/elasticsearch/bulk/BulkClient.java deleted file mode 100644 index d21837678..000000000 --- a/src/main/java/io/confluent/connect/elasticsearch/bulk/BulkClient.java +++ /dev/null @@ -1,27 +0,0 @@ -/* - * 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.connect.elasticsearch.bulk; - -import java.io.IOException; -import java.util.List; - -public interface BulkClient { - - B bulkRequest(List batch); - - BulkResponse execute(B req) throws IOException; - -} diff --git a/src/main/java/io/confluent/connect/elasticsearch/bulk/BulkProcessor.java b/src/main/java/io/confluent/connect/elasticsearch/bulk/BulkProcessor.java deleted file mode 100644 index 34239f7c7..000000000 --- a/src/main/java/io/confluent/connect/elasticsearch/bulk/BulkProcessor.java +++ /dev/null @@ -1,667 +0,0 @@ -/* - * 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.connect.elasticsearch.bulk; - -import io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig; -import io.confluent.connect.elasticsearch.LogContext; -import io.confluent.connect.elasticsearch.RetryUtil; - -import io.searchbox.core.BulkResult.BulkResultItem; -import java.util.Collections; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; -import org.apache.kafka.common.config.ConfigDef; -import org.apache.kafka.common.utils.Time; -import org.apache.kafka.connect.errors.ConnectException; -import org.apache.kafka.connect.sink.ErrantRecordReporter; -import org.apache.kafka.connect.sink.SinkRecord; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.ArrayDeque; -import java.util.ArrayList; -import java.util.Deque; -import java.util.List; -import java.util.Locale; -import java.util.concurrent.Callable; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; -import java.util.concurrent.ThreadFactory; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.atomic.AtomicReference; - -/** - * @param record type - * @param bulk request type - */ -public class BulkProcessor { - - private static final Logger log = LoggerFactory.getLogger(BulkProcessor.class); - - private static final AtomicLong BATCH_ID_GEN = new AtomicLong(); - - private final Time time; - private final BulkClient bulkClient; - private final int maxBufferedRecords; - private final int batchSize; - private final long lingerMs; - private final int maxRetries; - private final long retryBackoffMs; - private final BehaviorOnMalformedDoc behaviorOnMalformedDoc; - private final ErrantRecordReporter reporter; - - private final Thread farmer; - private final ExecutorService executor; - - // thread-safe state, can be mutated safely without synchronization, - // but may be part of synchronized(this) wait() conditions so need to notifyAll() on changes - private volatile boolean stopRequested = false; - private volatile boolean flushRequested = false; - private final AtomicReference error = new AtomicReference<>(); - - // shared state, synchronized on (this), may be part of wait() conditions so need notifyAll() on - // changes - private final Deque unsentRecords; - protected final ConcurrentMap recordsToReportOnError; // visible for tests - private int inFlightRecords = 0; - private final LogContext logContext = new LogContext(); - - public BulkProcessor( - Time time, - BulkClient bulkClient, - int maxBufferedRecords, - int maxInFlightRequests, - int batchSize, - long lingerMs, - int maxRetries, - long retryBackoffMs, - BehaviorOnMalformedDoc behaviorOnMalformedDoc, - ErrantRecordReporter reporter - ) { - this.time = time; - this.bulkClient = bulkClient; - this.maxBufferedRecords = maxBufferedRecords; - this.batchSize = batchSize; - this.lingerMs = lingerMs; - this.maxRetries = maxRetries; - this.retryBackoffMs = retryBackoffMs; - this.behaviorOnMalformedDoc = behaviorOnMalformedDoc; - this.reporter = reporter; - - unsentRecords = new ArrayDeque<>(maxBufferedRecords); - recordsToReportOnError = reporter != null - ? new ConcurrentHashMap<>(maxBufferedRecords) - : null; - - final ThreadFactory threadFactory = makeThreadFactory(); - farmer = threadFactory.newThread(farmerTask()); - executor = Executors.newFixedThreadPool(maxInFlightRequests, threadFactory); - } - - private ThreadFactory makeThreadFactory() { - final AtomicInteger threadCounter = new AtomicInteger(); - final Thread.UncaughtExceptionHandler uncaughtExceptionHandler = - new Thread.UncaughtExceptionHandler() { - @Override - public void uncaughtException(Thread t, Throwable e) { - log.error("Uncaught exception in BulkProcessor thread {}", t, e); - failAndStop(e); - } - }; - return new ThreadFactory() { - @Override - public Thread newThread(Runnable r) { - final int threadId = threadCounter.getAndIncrement(); - final int objId = System.identityHashCode(this); - final Thread t = new BulkProcessorThread(logContext, r, objId, threadId); - t.setDaemon(true); - t.setUncaughtExceptionHandler(uncaughtExceptionHandler); - return t; - } - }; - } - - // visible for testing - Runnable farmerTask() { - return () -> { - try (LogContext context = logContext.create("Farmer1")) { - log.debug("Starting farmer task"); - try { - List> futures = new ArrayList<>(); - while (!stopRequested) { - // submitBatchWhenReady waits for lingerMs so we won't spin here unnecessarily - futures.add(submitBatchWhenReady()); - - // after we submit, look at any previous futures that were completed and call get() on - // them so that exceptions are propagated. - List> unfinishedFutures = new ArrayList<>(); - for (Future f : futures) { - if (f.isDone()) { - BulkResponse resp = f.get(); - log.debug("Bulk request completed with status {}", resp); - } else { - unfinishedFutures.add(f); - } - } - log.debug("Processing next batch with {} outstanding batch requests in flight", - unfinishedFutures.size()); - futures = unfinishedFutures; - } - } catch (InterruptedException | ExecutionException e) { - throw new ConnectException(e); - } - log.debug("Finished farmer task"); - } - }; - } - - // Visible for testing - synchronized Future submitBatchWhenReady() throws InterruptedException { - for (long waitStartTimeMs = time.milliseconds(), elapsedMs = 0; - !stopRequested && !canSubmit(elapsedMs); - elapsedMs = time.milliseconds() - waitStartTimeMs) { - // when linger time has already elapsed, we still have to ensure the other submission - // conditions hence the wait(0) in that case - wait(Math.max(0, lingerMs - elapsedMs)); - } - - // at this point, either stopRequested or canSubmit - return stopRequested - ? CompletableFuture.completedFuture( - BulkResponse.failure( - false, - "request not submitted during shutdown", - Collections.emptyMap() - ) - ) - : submitBatch(); - } - - private synchronized Future submitBatch() { - final int numUnsentRecords = unsentRecords.size(); - assert numUnsentRecords > 0; - final int batchableSize = Math.min(batchSize, numUnsentRecords); - final List batch = new ArrayList<>(batchableSize); - for (int i = 0; i < batchableSize; i++) { - batch.add(unsentRecords.removeFirst()); - } - inFlightRecords += batchableSize; - log.debug( - "Submitting batch of {} records; {} unsent and {} total in-flight records", - batchableSize, - numUnsentRecords, - inFlightRecords - ); - return executor.submit(new BulkTask(batch)); - } - - /** - * Submission is possible when there are unsent records and: - *

    - *
  • flush is called, or
  • - *
  • the linger timeout passes, or
  • - *
  • there are sufficient records to fill a batch
  • - *
- */ - private synchronized boolean canSubmit(long elapsedMs) { - return !unsentRecords.isEmpty() - && (flushRequested || elapsedMs >= lingerMs || unsentRecords.size() >= batchSize); - } - - /** - * Start concurrently creating and sending batched requests using the client. - */ - public void start() { - farmer.start(); - } - - /** - * Initiate shutdown. - * - *

Pending buffered records are not automatically flushed, so call {@link #flush(long)} before - * this method if this is desirable. - */ - public void stop() { - log.debug("stop"); - stopRequested = true; // this stops the farmer task - synchronized (this) { - // shutdown the pool under synchronization to avoid rejected submissions - executor.shutdown(); - notifyAll(); - } - } - - /** - * Block upto {@code timeoutMs} till shutdown is complete. - * - *

This should only be called after a previous {@link #stop()} invocation. - */ - public void awaitStop(long timeoutMs) { - assert stopRequested; - try { - if (!executor.awaitTermination(timeoutMs, TimeUnit.MILLISECONDS)) { - throw new ConnectException("Timed-out waiting for executor termination"); - } - } catch (InterruptedException e) { - throw new ConnectException(e); - } finally { - executor.shutdownNow(); - } - } - - /** - * @return whether {@link #stop()} has been requested - */ - public boolean isStopping() { - return stopRequested; - } - - /** - * @return whether any task failed with an error - */ - public boolean isFailed() { - return error.get() != null; - } - - /** - * @return {@link #isTerminal()} or {@link #isFailed()} - */ - public boolean isTerminal() { - return isStopping() || isFailed(); - } - - /** - * Throw a {@link ConnectException} if {@link #isStopping()}. - */ - public void throwIfStopping() { - if (stopRequested) { - throw new ConnectException("Stopping"); - } - } - - /** - * Throw the relevant {@link ConnectException} if {@link #isFailed()}. - */ - public void throwIfFailed() { - if (isFailed()) { - throw error.get(); - } - } - - /** - * {@link #throwIfFailed()} and {@link #throwIfStopping()} - */ - public void throwIfTerminal() { - throwIfFailed(); - throwIfStopping(); - } - - /** - * Add a record, may block upto {@code timeoutMs} if at capacity with respect to - * {@code maxBufferedRecords}. - * - *

If any task has failed prior to or while blocked in the add, or if the timeout expires - * while blocked, {@link ConnectException} will be thrown. - */ - public synchronized void add(R record, SinkRecord original, long timeoutMs) { - throwIfTerminal(); - - int numBufferedRecords = bufferedRecords(); - if (numBufferedRecords >= maxBufferedRecords) { - log.trace( - "Buffer full at {} records, so waiting up to {} ms before adding", - numBufferedRecords, - timeoutMs - ); - final long addStartTimeMs = time.milliseconds(); - for (long elapsedMs = time.milliseconds() - addStartTimeMs; - !isTerminal() && elapsedMs < timeoutMs && bufferedRecords() >= maxBufferedRecords; - elapsedMs = time.milliseconds() - addStartTimeMs) { - try { - wait(timeoutMs - elapsedMs); - } catch (InterruptedException e) { - throw new ConnectException(e); - } - } - throwIfTerminal(); - if (bufferedRecords() >= maxBufferedRecords) { - throw new ConnectException("Add timeout expired before buffer availability"); - } - log.debug( - "Adding record to queue after waiting {} ms", - time.milliseconds() - addStartTimeMs - ); - } else { - log.trace("Adding record to queue"); - } - - unsentRecords.addLast(record); - addRecordToReport(record, original); - notifyAll(); - } - - /** - * Request a flush and block upto {@code timeoutMs} until all pending records have been flushed. - * - *

If any task has failed prior to or during the flush, {@link ConnectException} will be - * thrown with that error. - */ - public void flush(long timeoutMs) { - final long flushStartTimeMs = time.milliseconds(); - try { - flushRequested = true; - synchronized (this) { - notifyAll(); - for (long elapsedMs = time.milliseconds() - flushStartTimeMs; - !isTerminal() && elapsedMs < timeoutMs && bufferedRecords() > 0; - elapsedMs = time.milliseconds() - flushStartTimeMs) { - wait(timeoutMs - elapsedMs); - } - throwIfTerminal(); - if (bufferedRecords() > 0) { - throw new ConnectException("Flush timeout expired with unflushed records: " - + bufferedRecords()); - } - } - } catch (InterruptedException e) { - throw new ConnectException(e); - } finally { - flushRequested = false; - } - log.debug("Flushed bulk processor (total time={} ms)", time.milliseconds() - flushStartTimeMs); - } - - private void addRecordToReport(R record, SinkRecord original) { - if (reporter != null) { - // avoid unnecessary operations if not using the reporter - recordsToReportOnError.put(record, original); - } - } - - private void removeReportedRecords(List batch) { - if (reporter != null) { - // avoid unnecessary operations if not using the reporter - recordsToReportOnError.keySet().removeAll(batch); - } - } - - private static final class BulkProcessorThread extends Thread { - - private final LogContext parentContext; - private final int threadId; - - public BulkProcessorThread( - LogContext parentContext, - Runnable target, - int objId, - int threadId - ) { - super(target, String.format("BulkProcessor@%d-%d", objId, threadId)); - this.parentContext = parentContext; - this.threadId = threadId; - } - - @Override - public void run() { - try (LogContext context = parentContext.create("Thread" + threadId)) { - super.run(); - } - } - } - - private final class BulkTask implements Callable { - - final long batchId = BATCH_ID_GEN.incrementAndGet(); - - final List batch; - - BulkTask(List batch) { - this.batch = batch; - } - - @Override - public BulkResponse call() throws Exception { - final BulkResponse rsp; - try { - rsp = execute(); - } catch (Exception e) { - failAndStop(e); - throw e; - } - onBatchCompletion(batch.size()); - return rsp; - } - - private BulkResponse execute() throws Exception { - final long startTime = System.currentTimeMillis(); - final B bulkReq; - try { - bulkReq = bulkClient.bulkRequest(batch); - } catch (Exception e) { - log.error( - "Failed to create bulk request from batch {} of {} records", - batchId, - batch.size(), - e - ); - removeReportedRecords(batch); - throw e; - } - final int maxAttempts = maxRetries + 1; - for (int attempts = 1, retryAttempts = 0; true; ++attempts, ++retryAttempts) { - boolean retriable = true; - try { - log.trace("Executing batch {} of {} records with attempt {}/{}", - batchId, batch.size(), attempts, maxAttempts); - final BulkResponse bulkRsp = bulkClient.execute(bulkReq); - if (bulkRsp.isSucceeded()) { - if (log.isDebugEnabled()) { - log.debug( - "Completed batch {} of {} records with attempt {}/{} in {} ms", - batchId, - batch.size(), - attempts, - maxAttempts, - System.currentTimeMillis() - startTime - ); - } - removeReportedRecords(batch); - return bulkRsp; - } else if (responseContainsMalformedDocError(bulkRsp)) { - retriable = bulkRsp.isRetriable(); - handleMalformedDoc(bulkRsp); - if (reporter != null) { - for (R record : batch) { - SinkRecord original = recordsToReportOnError.get(record); - BulkResultItem result = bulkRsp.failedRecords.get(record); - String error = result != null ? result.error : null; - if (error != null && original != null) { - reporter.report( - original, new ReportingException("Bulk request failed: " + error) - ); - } - } - } - removeReportedRecords(batch); - return bulkRsp; - } else { - // for all other errors, throw the error up - retriable = bulkRsp.isRetriable(); - throw new ConnectException("Bulk request failed: " + bulkRsp.getErrorInfo()); - } - } catch (Exception e) { - if (retriable && attempts < maxAttempts) { - long sleepTimeMs = RetryUtil.computeRandomRetryWaitTimeInMillis(retryAttempts, - retryBackoffMs); - log.warn("Failed to execute batch {} of {} records with attempt {}/{}, " - + "will attempt retry after {} ms. Failure reason: {}", - batchId, batch.size(), attempts, maxAttempts, sleepTimeMs, e.getMessage()); - time.sleep(sleepTimeMs); - if (Thread.interrupted()) { - log.error( - "Retrying batch {} of {} records interrupted after attempt {}/{}", - batchId, batch.size(), attempts, maxAttempts, e); - removeReportedRecords(batch); - throw e; - } - } else { - log.error("Failed to execute batch {} of {} records after total of {} attempt(s)", - batchId, batch.size(), attempts, e); - removeReportedRecords(batch); - throw e; - } - } - } - } - - private void handleMalformedDoc(BulkResponse bulkRsp) { - // if the elasticsearch request failed because of a malformed document, - // the behavior is configurable. - switch (behaviorOnMalformedDoc) { - case IGNORE: - log.debug("Encountered an illegal document error when executing batch {} of {}" - + " records. Ignoring and will not index record. Error was {}", - batchId, batch.size(), bulkRsp.getErrorInfo()); - return; - case WARN: - log.warn("Encountered an illegal document error when executing batch {} of {}" - + " records. Ignoring and will not index record. Error was {}", - batchId, batch.size(), bulkRsp.getErrorInfo()); - return; - case FAIL: - log.error("Encountered an illegal document error when executing batch {} of {}" - + " records. Error was {} (to ignore future records like this" - + " change the configuration property '{}' from '{}' to '{}').", - batchId, batch.size(), bulkRsp.getErrorInfo(), - ElasticsearchSinkConnectorConfig.BEHAVIOR_ON_MALFORMED_DOCS_CONFIG, - BehaviorOnMalformedDoc.FAIL, - BehaviorOnMalformedDoc.IGNORE); - throw new ConnectException("Bulk request failed: " + bulkRsp.getErrorInfo()); - default: - throw new RuntimeException(String.format( - "Unknown value for %s enum: %s", - BehaviorOnMalformedDoc.class.getSimpleName(), - behaviorOnMalformedDoc - )); - } - } - } - - private boolean responseContainsMalformedDocError(BulkResponse bulkRsp) { - return bulkRsp.getErrorInfo().contains("mapper_parsing_exception") - || bulkRsp.getErrorInfo().contains("illegal_argument_exception") - || bulkRsp.getErrorInfo().contains("action_request_validation_exception"); - } - - private synchronized void onBatchCompletion(int batchSize) { - inFlightRecords -= batchSize; - assert inFlightRecords >= 0; - notifyAll(); - } - - private void failAndStop(Throwable t) { - error.compareAndSet(null, toConnectException(t)); - stop(); - } - - /** - * @return sum of unsent and in-flight record counts - */ - public synchronized int bufferedRecords() { - return unsentRecords.size() + inFlightRecords; - } - - private static ConnectException toConnectException(Throwable t) { - if (t instanceof ConnectException) { - return (ConnectException) t; - } else { - return new ConnectException(t); - } - } - - public enum BehaviorOnMalformedDoc { - IGNORE, - WARN, - FAIL; - - public static final BehaviorOnMalformedDoc DEFAULT = FAIL; - - // Want values for "behavior.on.malformed.doc" property to be case-insensitive - public static final ConfigDef.Validator VALIDATOR = new ConfigDef.Validator() { - private final ConfigDef.ValidString validator = ConfigDef.ValidString.in(names()); - - @Override - public void ensureValid(String name, Object value) { - if (value instanceof String) { - value = ((String) value).toLowerCase(Locale.ROOT); - } - validator.ensureValid(name, value); - } - - // Overridden here so that ConfigDef.toEnrichedRst shows possible values correctly - @Override - public String toString() { - return validator.toString(); - } - - }; - - public static String[] names() { - BehaviorOnMalformedDoc[] behaviors = values(); - String[] result = new String[behaviors.length]; - - for (int i = 0; i < behaviors.length; i++) { - result[i] = behaviors[i].toString(); - } - - return result; - } - - @Override - public String toString() { - return name().toLowerCase(Locale.ROOT); - } - } - - /** - * Exception that hides the stack trace used for reporting errors from Elasticsearch - * (mapper_parser_exception, illegal_argument_exception, and action_request_validation_exception) - * resulting from bad records using the AK 2.6 reporter DLQ interface because the error did not - * come from that line due to multithreading. - */ - @SuppressWarnings("serial") - public static class ReportingException extends RuntimeException { - - public ReportingException(String message) { - super(message); - } - - /** - * This method is overriden to swallow the stack trace. - * - * @return Throwable - */ - @Override - public synchronized Throwable fillInStackTrace() { - return this; - } - } -} diff --git a/src/main/java/io/confluent/connect/elasticsearch/bulk/BulkRequest.java b/src/main/java/io/confluent/connect/elasticsearch/bulk/BulkRequest.java deleted file mode 100644 index 4bfc59765..000000000 --- a/src/main/java/io/confluent/connect/elasticsearch/bulk/BulkRequest.java +++ /dev/null @@ -1,29 +0,0 @@ -/* - * 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.connect.elasticsearch.bulk; - - -import java.util.List; - -/** - * BulkRequest is a marker interface for use with - * {@link io.confluent.connect.elasticsearch.ElasticsearchClient#createBulkRequest(List)} and - * {@link io.confluent.connect.elasticsearch.ElasticsearchClient#executeBulk(BulkRequest)}. - * Implementations will typically hold state comprised of instances of classes that are - * specific to the client library. - */ -public interface BulkRequest { -} diff --git a/src/main/java/io/confluent/connect/elasticsearch/bulk/BulkResponse.java b/src/main/java/io/confluent/connect/elasticsearch/bulk/BulkResponse.java deleted file mode 100644 index 6d5ca5902..000000000 --- a/src/main/java/io/confluent/connect/elasticsearch/bulk/BulkResponse.java +++ /dev/null @@ -1,91 +0,0 @@ -/* - * 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.connect.elasticsearch.bulk; - -import io.confluent.connect.elasticsearch.IndexableRecord; -import io.searchbox.core.BulkResult.BulkResultItem; -import java.util.Collections; -import java.util.Map; - -public class BulkResponse { - - private static final BulkResponse SUCCESS_RESPONSE = - new BulkResponse(true, false, "", Collections.emptyMap()); - - public final boolean succeeded; - public final boolean retriable; - public final String errorInfo; - public final Map failedRecords; - - /** - * Creates a BulkResponse. - * @param succeeded whether the bulk request was successful or not. - * @param retriable whether the bulk request should be retried. - * @param errorInfo the error string - * @param failedRecords map of failed records and their results. Never null. - */ - private BulkResponse( - boolean succeeded, - boolean retriable, - String errorInfo, - Map failedRecords - ) { - this.succeeded = succeeded; - this.retriable = retriable; - this.errorInfo = errorInfo; - this.failedRecords = failedRecords; - } - - public static BulkResponse success() { - return SUCCESS_RESPONSE; - } - - /** - * Creates a failed BulkResponse. - * @param retriable whether the error is retriable - * @param errorInfo the error string - * @param failedRecords map of failed records and their results. Never null. - * @return - */ - public static BulkResponse failure( - boolean retriable, - String errorInfo, - Map failedRecords - ) { - return new BulkResponse(false, retriable, errorInfo, failedRecords); - } - - public boolean isSucceeded() { - return succeeded; - } - - public boolean isRetriable() { - return retriable; - } - - public String getErrorInfo() { - return errorInfo; - } - - @Override - public String toString() { - return "BulkResponse{" - + "succeeded=" + succeeded - + ", retriable=" + retriable - + ", errorInfo='" + (errorInfo == null ? "" : errorInfo) + '\'' - + '}'; - } -} diff --git a/src/main/java/io/confluent/connect/elasticsearch/jest/JestBulkRequest.java b/src/main/java/io/confluent/connect/elasticsearch/jest/JestBulkRequest.java deleted file mode 100644 index e8270f219..000000000 --- a/src/main/java/io/confluent/connect/elasticsearch/jest/JestBulkRequest.java +++ /dev/null @@ -1,40 +0,0 @@ -/* - * 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.connect.elasticsearch.jest; - -import io.confluent.connect.elasticsearch.IndexableRecord; -import io.confluent.connect.elasticsearch.bulk.BulkRequest; -import io.searchbox.core.Bulk; -import java.util.List; - -public class JestBulkRequest implements BulkRequest { - - private final Bulk bulk; - private final List records; - - public JestBulkRequest(Bulk bulk, List records) { - this.bulk = bulk; - this.records = records; - } - - public Bulk getBulk() { - return bulk; - } - - public List records() { - return records; - } -} diff --git a/src/main/java/io/confluent/connect/elasticsearch/jest/JestElasticsearchClient.java b/src/main/java/io/confluent/connect/elasticsearch/jest/JestElasticsearchClient.java deleted file mode 100644 index e6a2a79d4..000000000 --- a/src/main/java/io/confluent/connect/elasticsearch/jest/JestElasticsearchClient.java +++ /dev/null @@ -1,716 +0,0 @@ -/* - * 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.connect.elasticsearch.jest; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.node.JsonNodeFactory; -import com.fasterxml.jackson.databind.node.ObjectNode; -import com.google.gson.JsonObject; -import io.confluent.connect.elasticsearch.ElasticsearchClient; -import io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig; -import io.confluent.connect.elasticsearch.IndexableRecord; -import io.confluent.connect.elasticsearch.Key; -import io.confluent.connect.elasticsearch.Mapping; -import io.confluent.connect.elasticsearch.RetryUtil; -import io.confluent.connect.elasticsearch.bulk.BulkRequest; -import io.confluent.connect.elasticsearch.bulk.BulkResponse; -import io.confluent.connect.elasticsearch.jest.actions.PortableJestCreateIndexBuilder; -import io.confluent.connect.elasticsearch.jest.actions.PortableJestGetMappingBuilder; -import io.confluent.connect.elasticsearch.jest.actions.PortableJestPutMappingBuilder; -import io.searchbox.action.Action; -import io.searchbox.action.BulkableAction; -import io.searchbox.client.JestClient; -import io.searchbox.client.JestClientFactory; -import io.searchbox.client.JestResult; -import io.searchbox.client.config.HttpClientConfig; -import io.searchbox.cluster.NodesInfo; -import io.searchbox.core.Bulk; -import io.searchbox.core.BulkResult; -import io.searchbox.core.BulkResult.BulkResultItem; -import io.searchbox.core.Delete; -import io.searchbox.core.DocumentResult; -import io.searchbox.core.Index; -import io.searchbox.core.Search; -import io.searchbox.core.SearchResult; -import io.searchbox.core.Update; -import io.searchbox.indices.CreateIndex; -import io.searchbox.indices.DeleteIndex; -import io.searchbox.indices.IndicesExists; -import io.searchbox.indices.Refresh; -import io.searchbox.indices.mapping.PutMapping; -import java.util.HashMap; -import java.util.Objects; -import javax.net.ssl.HostnameVerifier; -import org.apache.http.HttpHost; -import org.apache.http.auth.AuthScope; -import org.apache.http.auth.UsernamePasswordCredentials; -import org.apache.http.client.CredentialsProvider; -import org.apache.http.impl.client.BasicCredentialsProvider; -import org.apache.kafka.common.config.ConfigDef; -import org.apache.http.conn.ssl.SSLConnectionSocketFactory; -import org.apache.http.nio.conn.ssl.SSLIOSessionStrategy; -import org.apache.kafka.common.config.ConfigException; -import org.apache.kafka.common.network.Mode; -import org.apache.kafka.common.security.ssl.SslFactory; -import org.apache.kafka.common.utils.SystemTime; -import org.apache.kafka.common.utils.Time; -import org.apache.kafka.connect.data.Schema; -import org.apache.kafka.connect.errors.ConnectException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.HashSet; -import java.util.List; -import java.util.Locale; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.TimeUnit; -import java.util.stream.Collectors; -import javax.net.ssl.SSLContext; - -public class JestElasticsearchClient implements ElasticsearchClient { - private static final Logger log = LoggerFactory.getLogger(JestElasticsearchClient.class); - - // visible for testing - protected static final String MAPPER_PARSE_EXCEPTION - = "mapper_parse_exception"; - protected static final String VERSION_CONFLICT_ENGINE_EXCEPTION - = "version_conflict_engine_exception"; - protected static final String ALL_FIELD_PARAM - = "_all"; - protected static final String RESOURCE_ALREADY_EXISTS_EXCEPTION - = "resource_already_exists_exception"; - - private static final Logger LOG = LoggerFactory.getLogger(JestElasticsearchClient.class); - - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - - private final JestClient client; - private final Version version; - private long timeout; - private WriteMethod writeMethod = WriteMethod.DEFAULT; - - private final Set indexCache = new HashSet<>(); - - private int maxRetries; - private long retryBackoffMs; - private final Time time = new SystemTime(); - private int retryOnConflict; - - // visible for testing - public JestElasticsearchClient(JestClient client) { - try { - this.client = client; - this.version = getServerVersion(); - } catch (IOException e) { - throw new ConnectException( - "Couldn't start ElasticsearchSinkTask due to connection error:", - e - ); - } - } - - // visible for testing - public JestElasticsearchClient(String address) { - try { - JestClientFactory factory = new JestClientFactory(); - factory.setHttpClientConfig(new HttpClientConfig.Builder(address) - .multiThreaded(true) - .build() - ); - this.client = factory.getObject(); - this.version = getServerVersion(); - } catch (IOException e) { - throw new ConnectException( - "Couldn't start ElasticsearchSinkTask due to connection error:", - e - ); - } catch (ConfigException e) { - throw new ConnectException( - "Couldn't start ElasticsearchSinkTask due to configuration error:", - e - ); - } - } - - public JestElasticsearchClient(Map props) { - this(props, new JestClientFactory()); - } - - // visible for testing - protected JestElasticsearchClient(Map props, JestClientFactory factory) { - try { - ElasticsearchSinkConnectorConfig config = new ElasticsearchSinkConnectorConfig(props); - factory.setHttpClientConfig(getClientConfig(config)); - this.client = factory.getObject(); - this.version = getServerVersion(); - this.writeMethod = config.writeMethod(); - this.retryBackoffMs = config.retryBackoffMs(); - this.maxRetries = config.maxRetries(); - this.timeout = config.readTimeoutMs(); - this.retryOnConflict = config.maxInFlightRequests(); - } catch (IOException e) { - throw new ConnectException( - "Couldn't start ElasticsearchSinkTask due to connection error:", - e - ); - } catch (ConfigException e) { - throw new ConnectException( - "Couldn't start ElasticsearchSinkTask due to configuration error:", - e - ); - } - } - - // Visible for Testing - public static HttpClientConfig getClientConfig(ElasticsearchSinkConnectorConfig config) { - - Set addresses = config.connectionUrls(); - HttpClientConfig.Builder builder = - new HttpClientConfig.Builder(addresses) - .connTimeout(config.connectionTimeoutMs()) - .readTimeout(config.readTimeoutMs()) - .requestCompressionEnabled(config.compression()) - .defaultMaxTotalConnectionPerRoute(config.maxInFlightRequests()) - .maxConnectionIdleTime(config.maxIdleTimeMs(), TimeUnit.MILLISECONDS) - .multiThreaded(true); - if (config.isAuthenticatedConnection()) { - builder.defaultCredentials(config.username(), config.password().value()) - .preemptiveAuthTargetHosts( - addresses.stream().map(addr -> HttpHost.create(addr)).collect(Collectors.toSet()) - ); - } - - configureProxy(config, builder); - - if (config.secured()) { - log.info("Using secured connection to {}", addresses); - configureSslContext(builder, config); - } else { - log.info("Using unsecured connection to {}", addresses); - } - return builder.build(); - } - - private static void configureProxy( - ElasticsearchSinkConnectorConfig config, - HttpClientConfig.Builder builder - ) { - - if (config.isBasicProxyConfigured()) { - HttpHost proxy = new HttpHost(config.proxyHost(), config.proxyPort()); - builder.proxy(proxy); - - if (config.isProxyWithAuthenticationConfigured()) { - - CredentialsProvider credentialsProvider = new BasicCredentialsProvider(); - if (config.isAuthenticatedConnection()) { - config.connectionUrls().forEach( - addr -> - credentialsProvider.setCredentials( - new AuthScope(new HttpHost(addr)), - new UsernamePasswordCredentials(config.username(), config.password().value()) - ) - ); - } - - credentialsProvider.setCredentials( - new AuthScope(proxy), - new UsernamePasswordCredentials(config.proxyUsername(), config.proxyPassword().value()) - ); - - builder.credentialsProvider(credentialsProvider); - } - } - } - - private static void configureSslContext( - HttpClientConfig.Builder builder, - ElasticsearchSinkConnectorConfig config - ) { - SslFactory kafkaSslFactory = new SslFactory(Mode.CLIENT, null, false); - kafkaSslFactory.configure(config.sslConfigs()); - - SSLContext sslContext; - try { - // try AK <= 2.2 first - sslContext = - (SSLContext) SslFactory.class.getDeclaredMethod("sslContext").invoke(kafkaSslFactory); - log.debug("Using AK 2.2 SslFactory methods."); - } catch (Exception e) { - // must be running AK 2.3+ - log.debug("Could not find AK 2.2 SslFactory methods. Trying AK 2.3+ methods for SslFactory."); - - Object sslEngine; - try { - // try AK <= 2.6 second - sslEngine = SslFactory.class.getDeclaredMethod("sslEngineBuilder").invoke(kafkaSslFactory); - log.debug("Using AK 2.2-2.5 SslFactory methods."); - - } catch (Exception ex) { - // must be running AK 2.6+ - log.debug( - "Could not find Ak 2.3-2.5 methods for SslFactory." - + " Trying AK 2.6+ methods for SslFactory." - ); - try { - sslEngine = - SslFactory.class.getDeclaredMethod("sslEngineFactory").invoke(kafkaSslFactory); - log.debug("Using AK 2.6+ SslFactory methods."); - } catch (Exception exc) { - throw new ConnectException("Failed to find methods for SslFactory.", exc); - } - } - - try { - sslContext = - (SSLContext) sslEngine.getClass().getDeclaredMethod("sslContext").invoke(sslEngine); - } catch (Exception ex) { - throw new ConnectException("Could not create SSLContext.", ex); - } - } - - HostnameVerifier hostnameVerifier = config.shouldDisableHostnameVerification() - ? (hostname, session) -> true - : SSLConnectionSocketFactory.getDefaultHostnameVerifier(); - - // Sync calls - SSLConnectionSocketFactory sslSocketFactory = new SSLConnectionSocketFactory( - sslContext, hostnameVerifier); - builder.sslSocketFactory(sslSocketFactory); - - // Async calls - SSLIOSessionStrategy sessionStrategy = new SSLIOSessionStrategy(sslContext, hostnameVerifier); - builder.httpsIOSessionStrategy(sessionStrategy); - } - - // visible for testing - protected void setWriteMethod(WriteMethod writeMethod) { - this.writeMethod = writeMethod; - } - - /* - * This method uses the NodesInfo request to get the server version, which is expected to work - * with all versions of Elasticsearch. - */ - private Version getServerVersion() throws IOException { - // Default to newest version for forward compatibility - Version defaultVersion = Version.ES_V6; - - NodesInfo info = new NodesInfo.Builder().addCleanApiParameter("version").build(); - JsonObject result = client.execute(info).getJsonObject(); - if (result == null) { - LOG.warn("Couldn't get Elasticsearch version (result is null); assuming {}", defaultVersion); - return defaultVersion; - } - if (!result.has("nodes")) { - LOG.warn("Couldn't get Elasticsearch version from result {} (result has no nodes). " - + "Assuming {}.", result, defaultVersion); - return defaultVersion; - } - - checkForError(result); - - JsonObject nodesRoot = result.get("nodes").getAsJsonObject(); - if (nodesRoot == null || nodesRoot.entrySet().size() == 0) { - LOG.warn( - "Couldn't get Elasticsearch version (response nodesRoot is null or empty); assuming {}", - defaultVersion - ); - return defaultVersion; - } - - JsonObject nodeRoot = nodesRoot.entrySet().iterator().next().getValue().getAsJsonObject(); - if (nodeRoot == null) { - LOG.warn( - "Couldn't get Elasticsearch version (response nodeRoot is null); assuming {}", - defaultVersion - ); - return defaultVersion; - } - - String esVersion = nodeRoot.get("version").getAsString(); - Version version; - if (esVersion == null) { - version = defaultVersion; - LOG.warn( - "Couldn't get Elasticsearch version (response version is null); assuming {}", - version - ); - } else if (esVersion.startsWith("1.")) { - version = Version.ES_V1; - log.info("Detected Elasticsearch version is {}", version); - } else if (esVersion.startsWith("2.")) { - version = Version.ES_V2; - log.info("Detected Elasticsearch version is {}", version); - } else if (esVersion.startsWith("5.")) { - version = Version.ES_V5; - log.info("Detected Elasticsearch version is {}", version); - } else if (esVersion.startsWith("6.")) { - version = Version.ES_V6; - log.info("Detected Elasticsearch version is {}", version); - } else if (esVersion.startsWith("7.")) { - version = Version.ES_V7; - log.info("Detected Elasticsearch version is {}", version); - } else { - version = defaultVersion; - log.info("Detected unexpected Elasticsearch version {}, using {}", esVersion, version); - } - return version; - } - - private void checkForError(JsonObject result) { - if (result.has("error") && result.get("error").isJsonObject()) { - final JsonObject errorObject = result.get("error").getAsJsonObject(); - String errorType = errorObject.has("type") ? errorObject.get("type").getAsString() : ""; - String errorReason = errorObject.has("reason") ? errorObject.get("reason").getAsString() : ""; - throw new ConnectException("Couldn't connect to Elasticsearch, error: " - + errorType + ", reason: " + errorReason); - } - } - - public Version getVersion() { - return version; - } - - private boolean indexExists(String index) { - if (indexCache.contains(index)) { - return true; - } - Action action = new IndicesExists.Builder(index).build(); - try { - log.info("Index '{}' not found in local cache; checking for existence", index); - JestResult result = client.execute(action); - log.debug("Received response for checking existence of index '{}'", index); - boolean exists = result.isSucceeded(); - if (exists) { - indexCache.add(index); - log.info("Index '{}' exists in Elasticsearch; adding to local cache", index); - } else { - log.info("Index '{}' not found in Elasticsearch. Error message: {}", - index, result.getErrorMessage()); - } - return exists; - } catch (IOException e) { - throw new ConnectException(e); - } - } - - public void createIndices(Set indices) { - log.trace("Attempting to discover or create indexes in Elasticsearch: {}", indices); - for (String index : indices) { - if (!indexExists(index)) { - final int maxAttempts = maxRetries + 1; - int attempts = 1; - CreateIndex createIndex = - new PortableJestCreateIndexBuilder(index, version, timeout).build(); - boolean indexed = false; - while (!indexed) { - try { - createIndex(index, createIndex); - indexed = true; - } catch (ConnectException e) { - if (attempts < maxAttempts) { - long sleepTimeMs = RetryUtil.computeRandomRetryWaitTimeInMillis(attempts - 1, - retryBackoffMs); - log.warn("Failed to create index {} with attempt {}/{}, " - + "will attempt retry after {} ms. Failure reason: {}", - index, attempts, maxAttempts, sleepTimeMs, e.getMessage()); - time.sleep(sleepTimeMs); - } else { - throw e; - } - attempts++; - } - } - } - } - } - - private void createIndex(String index, CreateIndex createIndex) throws ConnectException { - try { - log.info("Requesting Elasticsearch create index '{}'", index); - JestResult result = client.execute(createIndex); - log.debug("Received response for request to create index '{}'", index); - if (!result.isSucceeded()) { - boolean exists = result.getErrorMessage().contains(RESOURCE_ALREADY_EXISTS_EXCEPTION) - || indexExists(index); - - // Check if index was created by another client - if (!exists) { - String msg = - result.getErrorMessage() != null ? ": " + result.getErrorMessage() : ""; - throw new ConnectException("Could not create index '" + index + "'" + msg); - } - log.info("Index '{}' exists in Elasticsearch; adding to local cache", index); - } else { - log.info("Index '{}' created in Elasticsearch; adding to local cache", index); - } - indexCache.add(index); - } catch (IOException e) { - throw new ConnectException(e); - } - } - - public void createMapping(String index, String type, Schema schema) throws IOException { - ObjectNode obj = JsonNodeFactory.instance.objectNode(); - obj.set(type, Mapping.inferMapping(this, schema)); - PutMapping putMapping = new PortableJestPutMappingBuilder(index, type, obj.toString(), version) - .build(); - log.info("Submitting put mapping (type={}) for index '{}' and schema {}", type, index, schema); - JestResult result = client.execute(putMapping); - if (!result.isSucceeded()) { - throw new ConnectException( - "Cannot create mapping " + obj + " -- " + result.getErrorMessage() - ); - } - log.info("Completed put mapping (type={}) for index '{}' and schema {}", type, index, schema); - } - - /** - * Get the JSON mapping for given index and type. Returns {@code null} if it does not exist. - */ - public JsonObject getMapping(String index, String type) throws IOException { - log.info("Get mapping (type={}) for index '{}'", type, index); - final JestResult result = client.execute( - new PortableJestGetMappingBuilder(version) - .addIndex(index) - .addType(type) - .build() - ); - final JsonObject indexRoot = result.getJsonObject().getAsJsonObject(index); - if (indexRoot == null) { - log.debug("Received null (root) mapping (type={}) for index '{}'", type, index); - return null; - } - final JsonObject mappingsJson = indexRoot.getAsJsonObject("mappings"); - if (mappingsJson == null) { - log.debug("Received null mapping (type={}) for index '{}'", type, index); - return null; - } - log.debug("Received mapping (type={}) for index '{}'", type, index); - return mappingsJson.getAsJsonObject(type); - } - - /** - * Delete all indexes in Elasticsearch (useful for test) - */ - // For testing purposes - public void deleteAll() throws IOException { - log.info("Request deletion of all indexes"); - final JestResult result = client.execute(new DeleteIndex - .Builder(ALL_FIELD_PARAM) - .build()); - if (result.isSucceeded()) { - log.info("Deletion of all indexes succeeded"); - } else { - String msg = result.getErrorMessage() != null ? ": " + result.getErrorMessage() : ""; - log.warn("Could not delete all indexes: {}", msg); - } - } - - /** - * Refresh all data in elasticsearch, making it available for search (useful for testing) - */ - // For testing purposes - public void refresh() throws IOException { - log.info("Request refresh"); - final JestResult result = client.execute( - new Refresh.Builder().build() - ); - if (result.isSucceeded()) { - log.info("Refresh completed"); - } else { - String msg = result.getErrorMessage() != null ? ": " + result.getErrorMessage() : ""; - log.warn("Could not refresh: {}", msg); - } - } - - public BulkRequest createBulkRequest(List batch) { - final Bulk.Builder builder = new Bulk.Builder(); - for (IndexableRecord record : batch) { - builder.addAction(toBulkableAction(record)); - } - return new JestBulkRequest(builder.build(), batch); - } - - // visible for testing - protected BulkableAction toBulkableAction(IndexableRecord record) { - // If payload is null, the record was a tombstone and we should delete from the index. - if (record.payload == null) { - return toDeleteRequest(record); - } - return writeMethod == WriteMethod.INSERT - ? toIndexRequest(record) - : toUpdateRequest(record); - } - - private Delete toDeleteRequest(IndexableRecord record) { - Delete.Builder req = new Delete.Builder(record.key.id) - .index(record.key.index) - .type(record.key.type); - - // TODO: Should version information be set here? - return req.build(); - } - - private Index toIndexRequest(IndexableRecord record) { - Index.Builder req = new Index.Builder(record.payload) - .index(record.key.index) - .type(record.key.type) - .id(record.key.id); - if (record.version != null) { - req.setParameter("version_type", "external").setParameter("version", record.version); - } - return req.build(); - } - - private Update toUpdateRequest(IndexableRecord record) { - String payload = "{\"doc\":" + record.payload - + ", \"doc_as_upsert\":true}"; - return new Update.Builder(payload) - .index(record.key.index) - .type(record.key.type) - .id(record.key.id) - .setParameter("retry_on_conflict", retryOnConflict) - .build(); - } - - public BulkResponse executeBulk(BulkRequest bulk) throws IOException { - final BulkResult result = client.execute(((JestBulkRequest) bulk).getBulk()); - - if (result.isSucceeded()) { - return BulkResponse.success(); - } - log.debug("Bulk request failed; collecting error(s)"); - - boolean retriable = true; - - final List versionConflicts = new ArrayList<>(); - final List errors = new ArrayList<>(); - - for (BulkResult.BulkResultItem item : result.getItems()) { - if (item.error != null) { - final ObjectNode parsedError = (ObjectNode) OBJECT_MAPPER.readTree(item.error); - final String errorType = parsedError.get("type").asText(""); - if ("version_conflict_engine_exception".equals(errorType)) { - versionConflicts.add(new Key(item.index, item.type, item.id)); - } else if ("mapper_parse_exception".equals(errorType)) { - retriable = false; - errors.add(item.error); - } else { - errors.add(item.error); - } - } - } - - if (!versionConflicts.isEmpty()) { - LOG.warn("Ignoring version conflicts for items: {}", versionConflicts); - if (errors.isEmpty()) { - // The only errors were version conflicts - return BulkResponse.success(); - } - } - - final String errorInfo = errors.isEmpty() ? result.getErrorMessage() : errors.toString(); - - Map failedResponses = new HashMap<>(); - List items = result.getItems(); - List records = ((JestBulkRequest) bulk).records(); - for (int i = 0; i < items.size() && i < records.size() ; i++) { - BulkResultItem item = items.get(i); - IndexableRecord record = records.get(i); - if (item.error != null && Objects.equals(item.id, record.key.id)) { - // sanity check matching IDs - failedResponses.put(record, item); - } - } - - if (items.size() != records.size()) { - log.error( - "Elasticsearch bulk response size ({}) does not correspond to records sent ({})", - ((JestBulkRequest) bulk).records().size(), - items.size() - ); - } - - return BulkResponse.failure(retriable, errorInfo, failedResponses); - } - - // For testing purposes - public JsonObject search(String query, String index, String type) throws IOException { - final Search.Builder search = new Search.Builder(query); - if (index != null) { - search.addIndex(index); - } - if (type != null) { - search.addType(type); - } - - log.info("Executing search on index '{}' (type={}): {}", index, type, query); - final SearchResult result = client.execute(search.build()); - if (result.isSucceeded()) { - log.info("Executing search succeeded: {}", result); - } else { - String msg = result.getErrorMessage() != null ? ": " + result.getErrorMessage() : ""; - log.warn("Failed to execute search: {}", msg); - } - return result.getJsonObject(); - } - - public void close() { - try { - log.debug("Closing Elasticsearch client"); - client.close(); - } catch (IOException e) { - LOG.error("Exception while closing the JEST client", e); - } - } - - public enum WriteMethod { - INSERT, - UPSERT, - ; - - public static final WriteMethod DEFAULT = INSERT; - public static final ConfigDef.Validator VALIDATOR = new ConfigDef.Validator() { - private final ConfigDef.ValidString validator = ConfigDef.ValidString.in(names()); - - @Override - public void ensureValid(String name, Object value) { - validator.ensureValid(name, value); - } - - // Overridden here so that ConfigDef.toEnrichedRst shows possible values correctly - @Override - public String toString() { - return "One of " + INSERT.toString() + " or " + UPSERT.toString(); - } - - }; - - public static String[] names() { - return new String[] {INSERT.toString(), UPSERT.toString()}; - } - - @Override - public String toString() { - return name().toLowerCase(Locale.ROOT); - } - } -} diff --git a/src/main/java/io/confluent/connect/elasticsearch/jest/actions/PortableJestCreateIndexBuilder.java b/src/main/java/io/confluent/connect/elasticsearch/jest/actions/PortableJestCreateIndexBuilder.java deleted file mode 100644 index dc980a82a..000000000 --- a/src/main/java/io/confluent/connect/elasticsearch/jest/actions/PortableJestCreateIndexBuilder.java +++ /dev/null @@ -1,52 +0,0 @@ -/* - * 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.connect.elasticsearch.jest.actions; - -import io.confluent.connect.elasticsearch.ElasticsearchClient.Version; -import io.searchbox.indices.CreateIndex; - -/** - * Portable Jest action builder, across ES versions, to create indexes. - * This builder add support for ES version 7 by keeping the type support still enabled, this is - * done by passing the include_type_name parameter. This parameter is no longer required with ES 8, - * as types should not be used anymore by the time of ES 8 release. - */ -public class PortableJestCreateIndexBuilder extends CreateIndex.Builder { - - public static final String INCLUDE_TYPE_NAME_PARAM = "include_type_name"; - public static final String TIMEOUT_PARAM = "timeout"; - - private final Version version; - private final long timeout; - - public PortableJestCreateIndexBuilder(String index, Version version, long timeout) { - super(index); - this.version = version; - this.timeout = timeout; - } - - @Override - public CreateIndex build() { - if (version.equals(Version.ES_V7)) { - setParameter(INCLUDE_TYPE_NAME_PARAM, true); - } - if (timeout > 0) { - setParameter(TIMEOUT_PARAM, timeout + "ms"); - } - - return super.build(); - } -} diff --git a/src/main/java/io/confluent/connect/elasticsearch/jest/actions/PortableJestGetMappingBuilder.java b/src/main/java/io/confluent/connect/elasticsearch/jest/actions/PortableJestGetMappingBuilder.java deleted file mode 100644 index b26fe833b..000000000 --- a/src/main/java/io/confluent/connect/elasticsearch/jest/actions/PortableJestGetMappingBuilder.java +++ /dev/null @@ -1,44 +0,0 @@ -/* - * 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.connect.elasticsearch.jest.actions; - -import io.confluent.connect.elasticsearch.ElasticsearchClient.Version; -import io.searchbox.indices.mapping.GetMapping; - -/** - * Portable Jest action builder to put a new mapping. - * This builder add support for ES version 7 by keeping the type support still enabled, this is - * done by passing the include_type_name parameter. This parameter is no longer required with ES 8, - * as types should not be used anymore by the time of ES 8 release. - */ -public class PortableJestGetMappingBuilder extends GetMapping.Builder { - - public static final String INCLUDE_TYPE_NAME_PARAM = "include_type_name"; - - private final Version version; - - public PortableJestGetMappingBuilder(Version version) { - this.version = version; - } - - @Override - public GetMapping build() { - if (version.equals(Version.ES_V7)) { - setParameter(INCLUDE_TYPE_NAME_PARAM, true); - } - return super.build(); - } -} diff --git a/src/main/java/io/confluent/connect/elasticsearch/jest/actions/PortableJestPutMappingBuilder.java b/src/main/java/io/confluent/connect/elasticsearch/jest/actions/PortableJestPutMappingBuilder.java deleted file mode 100644 index 5e08ef337..000000000 --- a/src/main/java/io/confluent/connect/elasticsearch/jest/actions/PortableJestPutMappingBuilder.java +++ /dev/null @@ -1,45 +0,0 @@ -/* - * 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.connect.elasticsearch.jest.actions; - -import io.confluent.connect.elasticsearch.ElasticsearchClient.Version; -import io.searchbox.indices.mapping.PutMapping; - -/** - * Portable Jest action builder to put a new mapping. - * This builder add support for ES version 7 by keeping the type support still enabled, this is - * done by passing the include_type_name parameter. This parameter is no longer required with ES 8, - * as types should not be used anymore by the time of ES 8 release. - */ -public class PortableJestPutMappingBuilder extends PutMapping.Builder { - - public static final String INCLUDE_TYPE_NAME_PARAM = "include_type_name"; - - private final Version version; - - public PortableJestPutMappingBuilder(String index, String type, Object source, Version version) { - super(index, type, source); - this.version = version; - } - - @Override - public PutMapping build() { - if (version.equals(Version.ES_V7)) { - setParameter(INCLUDE_TYPE_NAME_PARAM, true); - } - return super.build(); - } -} diff --git a/src/test/java/io/confluent/connect/elasticsearch/DataConverterTest.java b/src/test/java/io/confluent/connect/elasticsearch/DataConverterTest.java index 469eaa023..6d46e6555 100644 --- a/src/test/java/io/confluent/connect/elasticsearch/DataConverterTest.java +++ b/src/test/java/io/confluent/connect/elasticsearch/DataConverterTest.java @@ -15,9 +15,12 @@ package io.confluent.connect.elasticsearch; +import io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.BehaviorOnNullValues; import org.apache.kafka.connect.data.*; import org.apache.kafka.connect.errors.DataException; import org.apache.kafka.connect.sink.SinkRecord; +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexRequest; import org.junit.Before; import org.junit.Test; @@ -29,8 +32,8 @@ import java.util.List; import java.util.Map; -import static io.confluent.connect.elasticsearch.DataConverter.BehaviorOnNullValues; - +import static io.confluent.connect.elasticsearch.DataConverter.MAP_KEY; +import static io.confluent.connect.elasticsearch.DataConverter.MAP_VALUE; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; import static org.junit.Assert.fail; @@ -38,23 +41,27 @@ public class DataConverterTest { private DataConverter converter; + private Map props; + private String key; private String topic; private int partition; private long offset; private String index; - private String type; private Schema schema; @Before public void setUp() { - converter = new DataConverter(true, BehaviorOnNullValues.DEFAULT); + props = ElasticsearchSinkConnectorConfigTest.addNecessaryProps(new HashMap<>()); + props.put(ElasticsearchSinkConnectorConfig.IGNORE_KEY_CONFIG, "true"); + props.put(ElasticsearchSinkConnectorConfig.COMPACT_MAP_ENTRIES_CONFIG, "true"); + + converter = new DataConverter(new ElasticsearchSinkConnectorConfig(props)); key = "key"; topic = "topic"; partition = 0; offset = 0; index = "index"; - type = "type"; schema = SchemaBuilder .struct() .name("struct") @@ -150,8 +157,8 @@ public void map() { assertEquals( SchemaBuilder.array( SchemaBuilder.struct().name(Schema.INT32_SCHEMA.type().name() + "-" + Decimal.LOGICAL_NAME) - .field(ElasticsearchSinkConnectorConstants.MAP_KEY, Schema.INT32_SCHEMA) - .field(ElasticsearchSinkConnectorConstants.MAP_VALUE, Schema.FLOAT64_SCHEMA) + .field(MAP_KEY, Schema.INT32_SCHEMA) + .field(MAP_VALUE, Schema.FLOAT64_SCHEMA) .build() ).build(), preProcessedSchema @@ -163,11 +170,11 @@ public void map() { assertEquals( new HashSet<>(Arrays.asList( new Struct(preProcessedSchema.valueSchema()) - .put(ElasticsearchSinkConnectorConstants.MAP_KEY, 1) - .put(ElasticsearchSinkConnectorConstants.MAP_VALUE, 0.02), + .put(MAP_KEY, 1) + .put(MAP_VALUE, 0.02), new Struct(preProcessedSchema.valueSchema()) - .put(ElasticsearchSinkConnectorConstants.MAP_KEY, 2) - .put(ElasticsearchSinkConnectorConstants.MAP_VALUE, 0.42) + .put(MAP_KEY, 2) + .put(MAP_VALUE, 0.42) )), new HashSet<>((List) converter.preProcessValue(origValue, origSchema, preProcessedSchema)) ); @@ -194,14 +201,15 @@ public void stringKeyedMapNonCompactFormat() { origValue.put("field2", 2); // Use the older non-compact format for map entries with string keys - converter = new DataConverter(false, BehaviorOnNullValues.DEFAULT); + props.put(ElasticsearchSinkConnectorConfig.COMPACT_MAP_ENTRIES_CONFIG, "false"); + converter = new DataConverter(new ElasticsearchSinkConnectorConfig(props)); Schema preProcessedSchema = converter.preProcessSchema(origSchema); assertEquals( SchemaBuilder.array( SchemaBuilder.struct().name(Schema.STRING_SCHEMA.type().name() + "-" + Schema.INT32_SCHEMA.type().name()) - .field(ElasticsearchSinkConnectorConstants.MAP_KEY, Schema.STRING_SCHEMA) - .field(ElasticsearchSinkConnectorConstants.MAP_VALUE, Schema.INT32_SCHEMA) + .field(MAP_KEY, Schema.STRING_SCHEMA) + .field(MAP_VALUE, Schema.INT32_SCHEMA) .build() ).build(), preProcessedSchema @@ -209,11 +217,11 @@ public void stringKeyedMapNonCompactFormat() { assertEquals( new HashSet<>(Arrays.asList( new Struct(preProcessedSchema.valueSchema()) - .put(ElasticsearchSinkConnectorConstants.MAP_KEY, "field1") - .put(ElasticsearchSinkConnectorConstants.MAP_VALUE, 1), + .put(MAP_KEY, "field1") + .put(MAP_VALUE, 1), new Struct(preProcessedSchema.valueSchema()) - .put(ElasticsearchSinkConnectorConstants.MAP_KEY, "field2") - .put(ElasticsearchSinkConnectorConstants.MAP_VALUE, 2) + .put(MAP_KEY, "field2") + .put(MAP_VALUE, 2) )), new HashSet<>((List) converter.preProcessValue(origValue, origSchema, preProcessedSchema)) ); @@ -228,7 +236,8 @@ public void stringKeyedMapCompactFormat() { origValue.put("field2", 2); // Use the newer compact format for map entries with string keys - converter = new DataConverter(true, BehaviorOnNullValues.DEFAULT); + props.put(ElasticsearchSinkConnectorConfig.COMPACT_MAP_ENTRIES_CONFIG, "true"); + converter = new DataConverter(new ElasticsearchSinkConnectorConfig(props)); Schema preProcessedSchema = converter.preProcessSchema(origSchema); assertEquals( SchemaBuilder.map(Schema.STRING_SCHEMA, Schema.INT32_SCHEMA).build(), @@ -272,7 +281,8 @@ public void optionalFieldsWithoutDefaults() { testOptionalFieldWithoutDefault(SchemaBuilder.struct().field("innerField", Schema.BOOLEAN_SCHEMA)); testOptionalFieldWithoutDefault(SchemaBuilder.map(Schema.STRING_SCHEMA, Schema.BOOLEAN_SCHEMA)); // Have to test maps with useCompactMapEntries set to true and set to false - converter = new DataConverter(false, BehaviorOnNullValues.DEFAULT); + props.put(ElasticsearchSinkConnectorConfig.COMPACT_MAP_ENTRIES_CONFIG, "false"); + converter = new DataConverter(new ElasticsearchSinkConnectorConfig(props)); testOptionalFieldWithoutDefault(SchemaBuilder.map(Schema.STRING_SCHEMA, Schema.BOOLEAN_SCHEMA)); } @@ -293,39 +303,104 @@ private void testOptionalFieldWithoutDefault( @Test public void ignoreOnNullValue() { - converter = new DataConverter(true, BehaviorOnNullValues.IGNORE); + + props.put(ElasticsearchSinkConnectorConfig.COMPACT_MAP_ENTRIES_CONFIG, "true"); + props.put(ElasticsearchSinkConnectorConfig.IGNORE_KEY_CONFIG, "false"); + props.put(ElasticsearchSinkConnectorConfig.IGNORE_SCHEMA_CONFIG, "false"); + props.put(ElasticsearchSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, BehaviorOnNullValues.IGNORE.name()); + converter = new DataConverter(new ElasticsearchSinkConnectorConfig(props)); SinkRecord sinkRecord = createSinkRecordWithValue(null); - assertNull(converter.convertRecord(sinkRecord, index, type, false, false)); + assertNull(converter.convertRecord(sinkRecord, index)); } @Test public void deleteOnNullValue() { - converter = new DataConverter(true, BehaviorOnNullValues.DELETE); + props.put(ElasticsearchSinkConnectorConfig.COMPACT_MAP_ENTRIES_CONFIG, "true"); + props.put(ElasticsearchSinkConnectorConfig.IGNORE_KEY_CONFIG, "false"); + props.put(ElasticsearchSinkConnectorConfig.IGNORE_SCHEMA_CONFIG, "false"); + props.put(ElasticsearchSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, BehaviorOnNullValues.DELETE.name()); + converter = new DataConverter(new ElasticsearchSinkConnectorConfig(props)); + SinkRecord sinkRecord = createSinkRecordWithValue(null); + DeleteRequest actualRecord = (DeleteRequest) converter.convertRecord(sinkRecord, index); + + assertEquals(key, actualRecord.id()); + assertEquals(index, actualRecord.index()); + assertEquals(sinkRecord.kafkaOffset(), actualRecord.version()); + } + + @Test + public void externalVersionHeaderOnDelete() { + String externalVersionHeader = "version"; + long expectedExternalVersion = 123l; + + props.put(ElasticsearchSinkConnectorConfig.COMPACT_MAP_ENTRIES_CONFIG, "true"); + props.put(ElasticsearchSinkConnectorConfig.IGNORE_KEY_CONFIG, "false"); + props.put(ElasticsearchSinkConnectorConfig.IGNORE_SCHEMA_CONFIG, "false"); + props.put(ElasticsearchSinkConnectorConfig.EXTERNAL_VERSION_HEADER_CONFIG, externalVersionHeader); + props.put(ElasticsearchSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, BehaviorOnNullValues.DELETE.name()); + converter = new DataConverter(new ElasticsearchSinkConnectorConfig(props)); SinkRecord sinkRecord = createSinkRecordWithValue(null); - IndexableRecord expectedRecord = createIndexableRecordWithPayload(null); - IndexableRecord actualRecord = converter.convertRecord(sinkRecord, index, type, false, false); + sinkRecord.headers().addLong(externalVersionHeader, expectedExternalVersion); + + DeleteRequest actualRecord = (DeleteRequest) converter.convertRecord(sinkRecord, index); - assertEquals(expectedRecord, actualRecord); + assertEquals(key, actualRecord.id()); + assertEquals(index, actualRecord.index()); + assertEquals(expectedExternalVersion, actualRecord.version()); + } + + @Test + public void externalVersionHeaderOnIndex() { + String externalVersionHeader = "version"; + long expectedExternalVersion = 123l; + + props.put(ElasticsearchSinkConnectorConfig.COMPACT_MAP_ENTRIES_CONFIG, "true"); + props.put(ElasticsearchSinkConnectorConfig.IGNORE_KEY_CONFIG, "false"); + props.put(ElasticsearchSinkConnectorConfig.IGNORE_SCHEMA_CONFIG, "false"); + props.put(ElasticsearchSinkConnectorConfig.EXTERNAL_VERSION_HEADER_CONFIG, externalVersionHeader); + props.put(ElasticsearchSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, BehaviorOnNullValues.DELETE.name()); + converter = new DataConverter(new ElasticsearchSinkConnectorConfig(props)); + + + Schema preProcessedSchema = converter.preProcessSchema(schema); + Struct struct = new Struct(preProcessedSchema).put("string", "myValue"); + SinkRecord sinkRecord = createSinkRecordWithValue(struct); + sinkRecord.headers().addLong(externalVersionHeader, expectedExternalVersion); + + IndexRequest actualRecord = (IndexRequest) converter.convertRecord(sinkRecord, index); + + assertEquals(key, actualRecord.id()); + assertEquals(index, actualRecord.index()); + assertEquals(expectedExternalVersion, actualRecord.version()); } @Test public void ignoreDeleteOnNullValueWithNullKey() { - converter = new DataConverter(true, BehaviorOnNullValues.DELETE); + props.put(ElasticsearchSinkConnectorConfig.COMPACT_MAP_ENTRIES_CONFIG, "true"); + props.put(ElasticsearchSinkConnectorConfig.IGNORE_KEY_CONFIG, "false"); + props.put(ElasticsearchSinkConnectorConfig.IGNORE_SCHEMA_CONFIG, "false"); + props.put(ElasticsearchSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, BehaviorOnNullValues.DELETE.name()); + converter = new DataConverter(new ElasticsearchSinkConnectorConfig(props)); + key = null; SinkRecord sinkRecord = createSinkRecordWithValue(null); - assertNull(converter.convertRecord(sinkRecord, index, type, false, false)); + assertNull(converter.convertRecord(sinkRecord, index)); } @Test public void failOnNullValue() { - converter = new DataConverter(true, BehaviorOnNullValues.FAIL); + props.put(ElasticsearchSinkConnectorConfig.COMPACT_MAP_ENTRIES_CONFIG, "true"); + props.put(ElasticsearchSinkConnectorConfig.IGNORE_KEY_CONFIG, "false"); + props.put(ElasticsearchSinkConnectorConfig.IGNORE_SCHEMA_CONFIG, "false"); + props.put(ElasticsearchSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG, BehaviorOnNullValues.FAIL.name()); + converter = new DataConverter(new ElasticsearchSinkConnectorConfig(props)); SinkRecord sinkRecord = createSinkRecordWithValue(null); try { - converter.convertRecord(sinkRecord, index, type, false, false); + converter.convertRecord(sinkRecord, index); fail("should fail on null-valued record with behaviorOnNullValues = FAIL"); } catch (DataException e) { // expected @@ -335,9 +410,4 @@ public void failOnNullValue() { public SinkRecord createSinkRecordWithValue(Object value) { return new SinkRecord(topic, partition, Schema.STRING_SCHEMA, key, schema, value, offset); } - - public IndexableRecord createIndexableRecordWithPayload(String payload) { - return new IndexableRecord(new Key(index, type, key), payload, offset); - } - } diff --git a/src/test/java/io/confluent/connect/elasticsearch/ElasticsearchClientTest.java b/src/test/java/io/confluent/connect/elasticsearch/ElasticsearchClientTest.java new file mode 100644 index 000000000..9dabb00e6 --- /dev/null +++ b/src/test/java/io/confluent/connect/elasticsearch/ElasticsearchClientTest.java @@ -0,0 +1,713 @@ +/* + * Copyright 2020 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.connect.elasticsearch; + +import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.BATCH_SIZE_CONFIG; +import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.BEHAVIOR_ON_MALFORMED_DOCS_CONFIG; +import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG; +import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.CONNECTION_PASSWORD_CONFIG; +import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.CONNECTION_URL_CONFIG; +import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.CONNECTION_USERNAME_CONFIG; +import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.IGNORE_KEY_CONFIG; +import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.LINGER_MS_CONFIG; +import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.MAX_BUFFERED_RECORDS_CONFIG; +import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.MAX_IN_FLIGHT_REQUESTS_CONFIG; +import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.MAX_RETRIES_CONFIG; +import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.RETRY_BACKOFF_MS_CONFIG; +import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.SECURITY_PROTOCOL_CONFIG; +import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.SSL_CONFIG_PREFIX; +import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.WRITE_METHOD_CONFIG; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThrows; +import static org.junit.Assert.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +import io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.BehaviorOnMalformedDoc; +import io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.BehaviorOnNullValues; +import io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.SecurityProtocol; +import io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.WriteMethod; +import io.confluent.connect.elasticsearch.helper.ElasticsearchContainer; +import io.confluent.connect.elasticsearch.helper.ElasticsearchHelperClient; +import io.confluent.connect.elasticsearch.helper.NetworkErrorContainer; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import org.apache.kafka.common.config.SslConfigs; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.errors.ConnectException; +import org.apache.kafka.connect.sink.ErrantRecordReporter; +import org.apache.kafka.connect.sink.SinkRecord; +import org.apache.kafka.test.TestUtils; +import org.elasticsearch.ElasticsearchStatusException; +import org.elasticsearch.action.DocWriteRequest; +import org.elasticsearch.action.bulk.BulkItemResponse; +import org.elasticsearch.index.VersionType; +import org.elasticsearch.search.SearchHit; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +public class ElasticsearchClientTest { + + private static final String INDEX = "index"; + private static final String ELASTIC_SUPERUSER_NAME = "elastic"; + private static final String ELASTIC_SUPERUSER_PASSWORD = "elastic"; + + private static ElasticsearchContainer container; + + private DataConverter converter; + private ElasticsearchHelperClient helperClient; + private ElasticsearchSinkConnectorConfig config; + private Map props; + + @BeforeClass + public static void setupBeforeAll() { + container = ElasticsearchContainer.fromSystemProperties(); + container.start(); + } + + @AfterClass + public static void cleanupAfterAll() { + container.close(); + } + + @Before + public void setup() { + props = ElasticsearchSinkConnectorConfigTest.addNecessaryProps(new HashMap<>()); + props.put(CONNECTION_URL_CONFIG, container.getConnectionUrl()); + props.put(IGNORE_KEY_CONFIG, "true"); + props.put(LINGER_MS_CONFIG, "1000"); + config = new ElasticsearchSinkConnectorConfig(props); + converter = new DataConverter(config); + helperClient = new ElasticsearchHelperClient(container.getConnectionUrl(), config); + } + + @After + public void cleanup() throws IOException { + if (helperClient != null && helperClient.indexExists(INDEX)){ + helperClient.deleteIndex(INDEX); + } + } + + @Test + public void testClose() { + ElasticsearchClient client = new ElasticsearchClient(config, null); + client.close(); + } + + @Test + public void testCloseFails() throws Exception { + props.put(BATCH_SIZE_CONFIG, "1"); + props.put(MAX_IN_FLIGHT_REQUESTS_CONFIG, "1"); + ElasticsearchClient client = new ElasticsearchClient(config, null) { + @Override + public void close() { + try { + if (!bulkProcessor.awaitClose(1, TimeUnit.MILLISECONDS)) { + throw new ConnectException("Failed to process all outstanding requests in time."); + } + } catch (InterruptedException e) {} + } + }; + + writeRecord(sinkRecord(0), client); + assertThrows( + "Failed to process all outstanding requests in time.", + ConnectException.class, + () -> client.close() + ); + waitUntilRecordsInES(1); + } + + @Test + public void testCreateIndex() throws IOException { + ElasticsearchClient client = new ElasticsearchClient(config, null); + assertFalse(helperClient.indexExists(INDEX)); + + client.createIndex(INDEX); + assertTrue(helperClient.indexExists(INDEX)); + client.close(); + } + + @Test + public void testDoesNotCreateAlreadyExistingIndex() throws IOException { + ElasticsearchClient client = new ElasticsearchClient(config, null); + assertFalse(helperClient.indexExists(INDEX)); + + assertTrue(client.createIndex(INDEX)); + assertTrue(helperClient.indexExists(INDEX)); + + assertFalse(client.createIndex(INDEX)); + assertTrue(helperClient.indexExists(INDEX)); + client.close(); + } + + @Test + public void testIndexExists() throws IOException { + ElasticsearchClient client = new ElasticsearchClient(config, null); + assertFalse(helperClient.indexExists(INDEX)); + + assertTrue(client.createIndex(INDEX)); + assertTrue(client.indexExists(INDEX)); + client.close(); + } + + @Test + public void testIndexDoesNotExist() throws IOException { + ElasticsearchClient client = new ElasticsearchClient(config, null); + assertFalse(helperClient.indexExists(INDEX)); + + assertFalse(client.indexExists(INDEX)); + client.close(); + } + + @Test + @SuppressWarnings("unchecked") + public void testCreateMapping() throws IOException { + ElasticsearchClient client = new ElasticsearchClient(config, null); + client.createIndex(INDEX); + + client.createMapping(INDEX, schema()); + + assertTrue(client.hasMapping(INDEX)); + + Map mapping = helperClient.getMapping(INDEX).sourceAsMap(); + assertTrue(mapping.containsKey("properties")); + Map props = (Map) mapping.get("properties"); + assertTrue(props.containsKey("offset")); + assertTrue(props.containsKey("another")); + Map offset = (Map) props.get("offset"); + assertEquals("integer", offset.get("type")); + assertEquals(0, offset.get("null_value")); + Map another = (Map) props.get("another"); + assertEquals("integer", another.get("type")); + assertEquals(0, another.get("null_value")); + client.close(); + } + + @Test + public void testHasMapping() { + ElasticsearchClient client = new ElasticsearchClient(config, null); + client.createIndex(INDEX); + + client.createMapping(INDEX, schema()); + + assertTrue(client.hasMapping(INDEX)); + client.close(); + } + + @Test + public void testDoesNotHaveMapping() { + ElasticsearchClient client = new ElasticsearchClient(config, null); + client.createIndex(INDEX); + + assertFalse(client.hasMapping(INDEX)); + client.close(); + } + + @Test + public void testBuffersCorrectly() throws Exception { + props.put(MAX_IN_FLIGHT_REQUESTS_CONFIG, "1"); + props.put(MAX_BUFFERED_RECORDS_CONFIG, "1"); + config = new ElasticsearchSinkConnectorConfig(props); + ElasticsearchClient client = new ElasticsearchClient(config, null); + client.createIndex(INDEX); + + writeRecord(sinkRecord(0), client); + assertEquals(1, client.numRecords.get()); + client.flush(); + + waitUntilRecordsInES(1); + assertEquals(1, helperClient.getDocCount(INDEX)); + assertEquals(0, client.numRecords.get()); + + writeRecord(sinkRecord(1), client); + assertEquals(1, client.numRecords.get()); + + // will block until the previous record is flushed + writeRecord(sinkRecord(2), client); + assertEquals(1, client.numRecords.get()); + + waitUntilRecordsInES(3); + client.close(); + } + + @Test + public void testFlush() throws Exception { + props.put(LINGER_MS_CONFIG, String.valueOf(TimeUnit.DAYS.toMillis(1))); + config = new ElasticsearchSinkConnectorConfig(props); + ElasticsearchClient client = new ElasticsearchClient(config, null); + client.createIndex(INDEX); + + writeRecord(sinkRecord(0), client); + assertEquals(0, helperClient.getDocCount(INDEX)); // should be empty before flush + + client.flush(); + + waitUntilRecordsInES(1); + assertEquals(1, helperClient.getDocCount(INDEX)); + client.close(); + } + + @Test + public void testIndexRecord() throws Exception { + ElasticsearchClient client = new ElasticsearchClient(config, null); + client.createIndex(INDEX); + + writeRecord(sinkRecord(0), client); + client.flush(); + + waitUntilRecordsInES(1); + assertEquals(1, helperClient.getDocCount(INDEX)); + client.close(); + } + + @Test + public void testDeleteRecord() throws Exception { + props.put(BEHAVIOR_ON_NULL_VALUES_CONFIG, BehaviorOnNullValues.DELETE.name()); + props.put(IGNORE_KEY_CONFIG, "false"); + config = new ElasticsearchSinkConnectorConfig(props); + converter = new DataConverter(config); + ElasticsearchClient client = new ElasticsearchClient(config, null); + client.createIndex(INDEX); + + writeRecord(sinkRecord("key0", 0), client); + writeRecord(sinkRecord("key1", 1), client); + client.flush(); + + waitUntilRecordsInES(2); + + // delete 1 + SinkRecord deleteRecord = new SinkRecord(INDEX, 0, Schema.STRING_SCHEMA, "key0", null, null, 3); + writeRecord(deleteRecord, client); + + waitUntilRecordsInES(1); + client.close(); + } + + @Test + public void testUpsertRecords() throws Exception { + props.put(WRITE_METHOD_CONFIG, WriteMethod.UPSERT.name()); + props.put(IGNORE_KEY_CONFIG, "false"); + config = new ElasticsearchSinkConnectorConfig(props); + converter = new DataConverter(config); + ElasticsearchClient client = new ElasticsearchClient(config, null); + client.createIndex(INDEX); + + writeRecord(sinkRecord("key0", 0), client); + writeRecord(sinkRecord("key1", 1), client); + client.flush(); + + waitUntilRecordsInES(2); + + // create modified record for upsert + Schema schema = SchemaBuilder + .struct() + .name("record") + .field("offset", SchemaBuilder.int32().defaultValue(0).build()) + .field("another", SchemaBuilder.int32().defaultValue(0).build()) + .build(); + + Struct value = new Struct(schema).put("offset", 2); + SinkRecord upsertRecord = new SinkRecord(INDEX, 0, Schema.STRING_SCHEMA, "key0", schema, value, 2); + Struct value2 = new Struct(schema).put("offset", 3); + SinkRecord upsertRecord2 = new SinkRecord(INDEX, 0, Schema.STRING_SCHEMA, "key0", schema, value2, 3); + + + // upsert 2, write another + writeRecord(upsertRecord, client); + writeRecord(upsertRecord2, client); + writeRecord(sinkRecord("key2", 4), client); + client.flush(); + + waitUntilRecordsInES(3); + for (SearchHit hit : helperClient.search(INDEX)) { + if (hit.getId().equals("key0")) { + assertEquals(3, hit.getSourceAsMap().get("offset")); + assertEquals(0, hit.getSourceAsMap().get("another")); + } + } + + client.close(); + } + + @Test + public void testIgnoreBadRecord() throws Exception { + props.put(BEHAVIOR_ON_MALFORMED_DOCS_CONFIG, BehaviorOnMalformedDoc.IGNORE.name()); + config = new ElasticsearchSinkConnectorConfig(props); + converter = new DataConverter(config); + + ElasticsearchClient client = new ElasticsearchClient(config, null); + client.createIndex(INDEX); + client.createMapping(INDEX, schema()); + + Schema schema = SchemaBuilder + .struct() + .name("record") + .field("not_mapped_field", SchemaBuilder.int32().defaultValue(0).build()) + .build(); + Struct value = new Struct(schema).put("not_mapped_field", 420); + SinkRecord badRecord = new SinkRecord(INDEX, 0, Schema.STRING_SCHEMA, "key", schema, value, 0); + + writeRecord(sinkRecord(0), client); + client.flush(); + + writeRecord(badRecord, client); + client.flush(); + + writeRecord(sinkRecord(1), client); + client.flush(); + + waitUntilRecordsInES(2); + assertEquals(2, helperClient.getDocCount(INDEX)); + client.close(); + } + + @Test(expected = ConnectException.class) + public void testFailOnBadRecord() throws Exception { + ElasticsearchClient client = new ElasticsearchClient(config, null); + client.createIndex(INDEX); + client.createMapping(INDEX, schema()); + + Schema schema = SchemaBuilder + .struct() + .name("record") + .field("offset", SchemaBuilder.bool().defaultValue(false).build()) + .build(); + Struct value = new Struct(schema).put("offset", false); + SinkRecord badRecord = new SinkRecord(INDEX, 0, Schema.STRING_SCHEMA, "key", schema, value, 0); + + writeRecord(sinkRecord(0), client); + client.flush(); + + waitUntilRecordsInES(1); + writeRecord(badRecord, client); + client.flush(); + + // consecutive index calls should cause exception + try { + for (int i = 0; i < 5; i++) { + writeRecord(sinkRecord(i + 1), client); + client.flush(); + waitUntilRecordsInES(i + 2); + } + } catch (ConnectException e) { + client.close(); + throw e; + } + } + + @Test + public void testRetryRecordsOnFailure() throws Exception { + props.put(LINGER_MS_CONFIG, "60000"); + props.put(BATCH_SIZE_CONFIG, "2"); + props.put(MAX_RETRIES_CONFIG, "100"); + props.put(RETRY_BACKOFF_MS_CONFIG, "1000"); + props.put(MAX_IN_FLIGHT_REQUESTS_CONFIG, "1"); + config = new ElasticsearchSinkConnectorConfig(props); + converter = new DataConverter(config); + + + // mock bulk processor to throw errors + ElasticsearchClient client = new ElasticsearchClient(config, null); + client.createIndex(INDEX); + + // bring down ES service + NetworkErrorContainer delay = new NetworkErrorContainer(container.getContainerName()); + delay.start(); + + // attempt a write + writeRecord(sinkRecord(0), client); + client.flush(); + + // keep the ES service down for a couple of timeouts + Thread.sleep(config.readTimeoutMs() * 4L); + + // bring up ES service + delay.stop(); + + waitUntilRecordsInES(1); + } + + @Test + public void testReporter() throws Exception { + props.put(IGNORE_KEY_CONFIG, "false"); + props.put(BEHAVIOR_ON_MALFORMED_DOCS_CONFIG, BehaviorOnMalformedDoc.IGNORE.name()); + config = new ElasticsearchSinkConnectorConfig(props); + converter = new DataConverter(config); + + ErrantRecordReporter reporter = mock(ErrantRecordReporter.class); + ElasticsearchClient client = new ElasticsearchClient(config, reporter); + client.createIndex(INDEX); + client.createMapping(INDEX, schema()); + + Schema schema = SchemaBuilder + .struct() + .name("record") + .field("offset", SchemaBuilder.bool().defaultValue(false).build()) + .build(); + Struct value = new Struct(schema).put("offset", false); + SinkRecord badRecord = new SinkRecord(INDEX, 0, Schema.STRING_SCHEMA, "key0", schema, value, 1); + + writeRecord(sinkRecord("key0", 0), client); + client.flush(); + waitUntilRecordsInES(1); + + writeRecord(badRecord, client); + client.flush(); + + // failed requests take a bit longer + for (int i = 2; i < 7; i++) { + writeRecord(sinkRecord("key" + i, i + 1), client); + client.flush(); + waitUntilRecordsInES(i); + } + + verify(reporter, times(1)).report(eq(badRecord), any(Throwable.class)); + client.close(); + } + + @Test + public void testReporterNotCalled() throws Exception { + ErrantRecordReporter reporter = mock(ErrantRecordReporter.class); + ElasticsearchClient client = new ElasticsearchClient(config, reporter); + client.createIndex(INDEX); + + writeRecord(sinkRecord(0), client); + writeRecord(sinkRecord(1), client); + writeRecord(sinkRecord(2), client); + client.flush(); + + waitUntilRecordsInES(3); + assertEquals(3, helperClient.getDocCount(INDEX)); + verify(reporter, never()).report(eq(sinkRecord(0)), any(Throwable.class)); + client.close(); + } + + + /** + * Cause a version conflict error. + * Assumes that Elasticsearch VersionType is 'EXTERNAL' for the records + * @param client The Elasticsearch client object to which to send records + */ + private void causeExternalVersionConflictError(ElasticsearchClient client) throws InterruptedException { + client.createIndex(INDEX); + + // Sequentially increase out record version (which comes from the offset) + writeRecord(sinkRecord(0), client); + writeRecord(sinkRecord(1), client); + writeRecord(sinkRecord(2), client); + client.flush(); + + // At the end of the day, it's just one record being overwritten + waitUntilRecordsInES(1); + + // Now duplicate the last and then the one before that + writeRecord(sinkRecord(2), client); + writeRecord(sinkRecord(1), client); + client.flush(); + } + + /** + * If the record version is set to VersionType.EXTERNAL (normal case for non-streaming), + * then same or less version number will throw a version conflict exception. + * @throws Exception will be thrown if the test fails + */ + @Test + public void testExternalVersionConflictReporterNotCalled() throws Exception { + props.put(IGNORE_KEY_CONFIG, "false"); + // Suppress asynchronous operations + props.put(MAX_IN_FLIGHT_REQUESTS_CONFIG, "1"); + config = new ElasticsearchSinkConnectorConfig(props); + converter = new DataConverter(config); + + ErrantRecordReporter reporter = mock(ErrantRecordReporter.class); + ElasticsearchClient client = new ElasticsearchClient(config, reporter); + + causeExternalVersionConflictError(client); + + // Make sure that no error was reported for either offset [1, 2] record(s) + verify(reporter, never()).report(eq(sinkRecord(1)), any(Throwable.class)); + verify(reporter, never()).report(eq(sinkRecord(2)), any(Throwable.class)); + client.close(); + } + + /** + * If the record version is set to VersionType.INTERNAL (normal case streaming/logging), + * then same or less version number will throw a version conflict exception. + * In this test, we are checking that the client function `handleResponse` + * properly reports an error for seeing the version conflict error along with + * VersionType of INTERNAL. We still actually cause the error via an external + * version conflict error, but flip the version type to internal before it is interpreted. + * @throws Exception will be thrown if the test fails + */ + @Test + public void testHandleResponseInternalVersionConflictReporterCalled() throws Exception { + props.put(IGNORE_KEY_CONFIG, "false"); + // Suppress asynchronous operations + props.put(MAX_IN_FLIGHT_REQUESTS_CONFIG, "1"); + config = new ElasticsearchSinkConnectorConfig(props); + converter = new DataConverter(config); + + ErrantRecordReporter reporter = mock(ErrantRecordReporter.class); + + // We will cause a version conflict error, but test that handleResponse() + // correctly reports the error when it interprets the version conflict as + // "INTERNAL" (version maintained by Elasticsearch) rather than + // "EXTERNAL" (version maintained by the connector as kafka offset) + ElasticsearchClient client = new ElasticsearchClient(config, reporter) { + protected void handleResponse(BulkItemResponse response, DocWriteRequest request, + long executionId) { + // Make it think it was an internal version conflict. + // Note that we don't make any attempt to reset the response version number, + // which will be -1 here. + request.versionType(VersionType.INTERNAL); + super.handleResponse(response, request, executionId); + } + }; + + causeExternalVersionConflictError(client); + + // Make sure that error was reported for either offset [1, 2] record(s) + verify(reporter, times(1)).report(eq(sinkRecord(1)), any(Throwable.class)); + verify(reporter, times(1)).report(eq(sinkRecord(2)), any(Throwable.class)); + client.close(); + } + + @Test + public void testNoVersionConflict() throws Exception { + props.put(IGNORE_KEY_CONFIG, "false"); + props.put(WRITE_METHOD_CONFIG, WriteMethod.UPSERT.name()); + config = new ElasticsearchSinkConnectorConfig(props); + converter = new DataConverter(config); + + ErrantRecordReporter reporter = mock(ErrantRecordReporter.class); + ErrantRecordReporter reporter2 = mock(ErrantRecordReporter.class); + ElasticsearchClient client = new ElasticsearchClient(config, reporter); + ElasticsearchClient client2 = new ElasticsearchClient(config, reporter2); + + client.createIndex(INDEX); + + writeRecord(sinkRecord(0), client); + writeRecord(sinkRecord(1), client2); + writeRecord(sinkRecord(2), client); + writeRecord(sinkRecord(3), client2); + writeRecord(sinkRecord(4), client); + writeRecord(sinkRecord(5), client2); + + waitUntilRecordsInES(1); + assertEquals(1, helperClient.getDocCount(INDEX)); + verify(reporter, never()).report(any(SinkRecord.class), any(Throwable.class)); + verify(reporter2, never()).report(any(SinkRecord.class), any(Throwable.class)); + client.close(); + client2.close(); + } + + @Test + public void testSsl() throws Exception { + container.close(); + container = ElasticsearchContainer.fromSystemProperties().withSslEnabled(true); + container.start(); + + String address = container.getConnectionUrl(false); + props.put(CONNECTION_URL_CONFIG, address); + props.put(CONNECTION_USERNAME_CONFIG, ELASTIC_SUPERUSER_NAME); + props.put(CONNECTION_PASSWORD_CONFIG, ELASTIC_SUPERUSER_PASSWORD); + props.put(SECURITY_PROTOCOL_CONFIG, SecurityProtocol.SSL.name()); + props.put(SSL_CONFIG_PREFIX + SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG, container.getKeystorePath()); + props.put(SSL_CONFIG_PREFIX + SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, container.getKeystorePassword()); + props.put(SSL_CONFIG_PREFIX + SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, container.getTruststorePath()); + props.put(SSL_CONFIG_PREFIX + SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, container.getTruststorePassword()); + props.put(SSL_CONFIG_PREFIX + SslConfigs.SSL_KEY_PASSWORD_CONFIG, container.getKeyPassword()); + config = new ElasticsearchSinkConnectorConfig(props); + converter = new DataConverter(config); + + ElasticsearchClient client = new ElasticsearchClient(config, null); + helperClient = new ElasticsearchHelperClient(address, config); + client.createIndex(INDEX); + + writeRecord(sinkRecord(0), client); + client.flush(); + + waitUntilRecordsInES(1); + assertEquals(1, helperClient.getDocCount(INDEX)); + client.close(); + helperClient = null; + + container.close(); + container = ElasticsearchContainer.fromSystemProperties(); + container.start(); + } + + @Test + public void testConnectionUrlExtraSlash() { + props.put(CONNECTION_URL_CONFIG, container.getConnectionUrl() + "/"); + config = new ElasticsearchSinkConnectorConfig(props); + ElasticsearchClient client = new ElasticsearchClient(config, null); + client.close(); + } + + private static Schema schema() { + return SchemaBuilder + .struct() + .name("record") + .field("offset", SchemaBuilder.int32().defaultValue(0).build()) + .field("another", SchemaBuilder.int32().defaultValue(0).build()) + .build(); + } + + private static SinkRecord sinkRecord(int offset) { + return sinkRecord("key", offset); + } + + private static SinkRecord sinkRecord(String key, int offset) { + Struct value = new Struct(schema()).put("offset", offset).put("another", offset + 1); + return new SinkRecord(INDEX, 0, Schema.STRING_SCHEMA, key, schema(), value, offset); + } + + private void waitUntilRecordsInES(int expectedRecords) throws InterruptedException { + TestUtils.waitForCondition( + () -> { + try { + return helperClient.getDocCount(INDEX) == expectedRecords; + } catch (ElasticsearchStatusException e) { + if (e.getMessage().contains("index_not_found_exception")) { + return false; + } + + throw e; + } + }, + TimeUnit.MINUTES.toMillis(1), + String.format("Could not find expected documents (%d) in time.", expectedRecords) + ); + } + + private void writeRecord(SinkRecord record, ElasticsearchClient client) { + client.index(record, converter.convertRecord(record, record.topic())); + } +} diff --git a/src/test/java/io/confluent/connect/elasticsearch/ElasticsearchSinkConnectorConfigTest.java b/src/test/java/io/confluent/connect/elasticsearch/ElasticsearchSinkConnectorConfigTest.java index 4950e9e31..77686844a 100644 --- a/src/test/java/io/confluent/connect/elasticsearch/ElasticsearchSinkConnectorConfigTest.java +++ b/src/test/java/io/confluent/connect/elasticsearch/ElasticsearchSinkConnectorConfigTest.java @@ -1,40 +1,31 @@ package io.confluent.connect.elasticsearch; -import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.CONNECTION_TIMEOUT_MS_CONFIG; -import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.CONNECTION_URL_CONFIG; -import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.IGNORE_KEY_CONFIG; -import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.PROXY_HOST_CONFIG; -import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.PROXY_PASSWORD_CONFIG; -import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.PROXY_PORT_CONFIG; -import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.PROXY_USERNAME_CONFIG; -import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.READ_TIMEOUT_MS_CONFIG; -import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.SECURITY_PROTOCOL_CONFIG; -import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.SSL_CONFIG_PREFIX; -import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.TYPE_NAME_CONFIG; +import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.*; +import static org.apache.kafka.common.config.SslConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; +import io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.SecurityProtocol; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.HashMap; +import java.util.Map; import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.config.SslConfigs; import org.apache.kafka.common.config.types.Password; import org.junit.Before; import org.junit.Test; -import java.util.HashMap; -import java.util.Map; - public class ElasticsearchSinkConnectorConfigTest { private Map props; @Before public void setup() { - props = new HashMap<>(); - props.put(TYPE_NAME_CONFIG, ElasticsearchSinkTestBase.TYPE); - props.put(CONNECTION_URL_CONFIG, "localhost"); - props.put(IGNORE_KEY_CONFIG, "true"); + props = addNecessaryProps(new HashMap<>()); } @Test @@ -49,6 +40,7 @@ public void testSetHttpTimeoutsConfig() { props.put(READ_TIMEOUT_MS_CONFIG, "10000"); props.put(CONNECTION_TIMEOUT_MS_CONFIG, "15000"); ElasticsearchSinkConnectorConfig config = new ElasticsearchSinkConnectorConfig(props); + assertEquals(config.readTimeoutMs(), 10000); assertEquals(config.connectionTimeoutMs(), 15000); } @@ -66,6 +58,7 @@ public void testSslConfigs() { props.put(SSL_CONFIG_PREFIX + SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, "/path2"); props.put(SSL_CONFIG_PREFIX + SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, "opensesame2"); ElasticsearchSinkConnectorConfig config = new ElasticsearchSinkConnectorConfig(props); + Map sslConfigs = config.sslConfigs(); assertTrue(sslConfigs.size() > 0); assertEquals( @@ -83,30 +76,31 @@ public void testSslConfigs() { @Test public void testSecured() { props.put(CONNECTION_URL_CONFIG, "http://host:9999"); - assertFalse(new ElasticsearchSinkConnectorConfig(props).secured()); + assertFalse(new ElasticsearchSinkConnectorConfig(props).isSslEnabled()); props.put(CONNECTION_URL_CONFIG, "https://host:9999"); - assertFalse(new ElasticsearchSinkConnectorConfig(props).secured()); + assertFalse(new ElasticsearchSinkConnectorConfig(props).isSslEnabled()); props.put(CONNECTION_URL_CONFIG, "http://host1:9992,https://host:9999"); - assertFalse(new ElasticsearchSinkConnectorConfig(props).secured()); + assertFalse(new ElasticsearchSinkConnectorConfig(props).isSslEnabled()); // Default behavior should be backwards compat props.put(CONNECTION_URL_CONFIG, "host1:9992"); - assertFalse(new ElasticsearchSinkConnectorConfig(props).secured()); + assertFalse(new ElasticsearchSinkConnectorConfig(props).isSslEnabled()); props.put(SECURITY_PROTOCOL_CONFIG, SecurityProtocol.SSL.name()); - assertTrue(new ElasticsearchSinkConnectorConfig(props).secured()); + assertTrue(new ElasticsearchSinkConnectorConfig(props).isSslEnabled()); props.put(SECURITY_PROTOCOL_CONFIG, SecurityProtocol.PLAINTEXT.name()); props.put(CONNECTION_URL_CONFIG, "https://host:9999"); - assertFalse(new ElasticsearchSinkConnectorConfig(props).secured()); + assertFalse(new ElasticsearchSinkConnectorConfig(props).isSslEnabled()); } @Test public void shouldAcceptValidBasicProxy() { props.put(PROXY_HOST_CONFIG, "proxy host"); ElasticsearchSinkConnectorConfig config = new ElasticsearchSinkConnectorConfig(props); + assertNotNull(config); assertTrue(config.isBasicProxyConfigured()); assertFalse(config.isProxyWithAuthenticationConfigured()); @@ -115,12 +109,18 @@ public void shouldAcceptValidBasicProxy() { @Test public void shouldAcceptValidProxyWithAuthentication() { props.put(PROXY_HOST_CONFIG, "proxy host"); + props.put(PROXY_PORT_CONFIG, "1010"); props.put(PROXY_USERNAME_CONFIG, "username"); props.put(PROXY_PASSWORD_CONFIG, "password"); ElasticsearchSinkConnectorConfig config = new ElasticsearchSinkConnectorConfig(props); + assertNotNull(config); assertTrue(config.isBasicProxyConfigured()); assertTrue(config.isProxyWithAuthenticationConfigured()); + assertEquals("proxy host", config.proxyHost()); + assertEquals(1010, config.proxyPort()); + assertEquals("username", config.proxyUsername()); + assertEquals("password", config.proxyPassword().value()); } @Test(expected = ConfigException.class) @@ -128,4 +128,74 @@ public void shouldNotAllowInvalidProxyPort() { props.put(PROXY_PORT_CONFIG, "-666"); new ElasticsearchSinkConnectorConfig(props); } + + @Test(expected = ConfigException.class) + public void shouldNotAllowInvalidUrl() { + props.put(CONNECTION_URL_CONFIG, ".com:/bbb/dfs,http://valid.com"); + new ElasticsearchSinkConnectorConfig(props); + } + + @Test(expected = ConfigException.class) + public void shouldNotAllowInvalidSecurityProtocol() { + props.put(SECURITY_PROTOCOL_CONFIG, "unsecure"); + new ElasticsearchSinkConnectorConfig(props); + } + + @Test + public void shouldDisableHostnameVerification() { + props.put(SSL_CONFIG_PREFIX + SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG, "https"); + ElasticsearchSinkConnectorConfig config = new ElasticsearchSinkConnectorConfig(props); + assertFalse(config.shouldDisableHostnameVerification()); + + props.put(SSL_CONFIG_PREFIX + SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG, ""); + config = new ElasticsearchSinkConnectorConfig(props); + assertTrue(config.shouldDisableHostnameVerification()); + + props.put(SSL_CONFIG_PREFIX + SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG, null); + config = new ElasticsearchSinkConnectorConfig(props); + assertFalse(config.shouldDisableHostnameVerification()); + } + + @Test(expected = ConfigException.class) + public void shouldNotAllowInvalidExtensionKeytab() { + props.put(KERBEROS_KEYTAB_PATH_CONFIG, "keytab.wrongextension"); + new ElasticsearchSinkConnectorConfig(props); + } + + @Test(expected = ConfigException.class) + public void shouldNotAllowNonExistingKeytab() { + props.put(KERBEROS_KEYTAB_PATH_CONFIG, "idontexist.keytab"); + new ElasticsearchSinkConnectorConfig(props); + } + + @Test + public void shouldAllowValidKeytab() throws IOException { + Path keytab = Files.createTempFile("iexist", ".keytab"); + props.put(KERBEROS_KEYTAB_PATH_CONFIG, keytab.toString()); + + new ElasticsearchSinkConnectorConfig(props); + + keytab.toFile().delete(); + } + + public static Map addNecessaryProps(Map props) { + if (props == null) { + props = new HashMap<>(); + } + props.put(ElasticsearchSinkConnectorConfig.CONNECTION_URL_CONFIG, "http://localhost:8080"); + return props; + } + @Test + public void testLogSensitiveData(){ + ElasticsearchSinkConnectorConfig config = new ElasticsearchSinkConnectorConfig(props); + assertFalse(config.shouldLogSensitiveData()); + + props.put(LOG_SENSITIVE_DATA_CONFIG, "true"); + config = new ElasticsearchSinkConnectorConfig(props); + assertTrue(config.shouldLogSensitiveData()); + + props.put(LOG_SENSITIVE_DATA_CONFIG, "false"); + config = new ElasticsearchSinkConnectorConfig(props); + assertFalse(config.shouldLogSensitiveData()); + } } diff --git a/src/test/java/io/confluent/connect/elasticsearch/ElasticsearchSinkConnectorTest.java b/src/test/java/io/confluent/connect/elasticsearch/ElasticsearchSinkConnectorTest.java new file mode 100644 index 000000000..a22dc656a --- /dev/null +++ b/src/test/java/io/confluent/connect/elasticsearch/ElasticsearchSinkConnectorTest.java @@ -0,0 +1,73 @@ +package io.confluent.connect.elasticsearch; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.kafka.common.config.Config; +import org.apache.kafka.connect.errors.ConnectException; +import org.apache.kafka.connect.sink.SinkConnector; +import org.junit.Before; +import org.junit.Test; + +public class ElasticsearchSinkConnectorTest { + + private ElasticsearchSinkConnector connector; + private Map settings; + + @Before + public void before() { + settings = ElasticsearchSinkConnectorConfigTest.addNecessaryProps(new HashMap<>()); + connector = new ElasticsearchSinkConnector(); + } + + @Test(expected = ConnectException.class) + public void shouldCatchInvalidConfigs() { + connector.start(new HashMap<>()); + } + + @Test + public void shouldGenerateValidTaskConfigs() { + connector.start(settings); + List> taskConfigs = connector.taskConfigs(2); + assertFalse("zero task configs provided", taskConfigs.isEmpty()); + for (Map taskConfig : taskConfigs) { + assertEquals(settings, taskConfig); + } + } + + @Test + public void shouldNotHaveNullConfigDef() { + // ConfigDef objects don't have an overridden equals() method; just make sure it's non-null + assertNotNull(connector.config()); + } + + @Test + public void shouldReturnConnectorType() { + assertTrue(SinkConnector.class.isAssignableFrom(connector.getClass())); + } + + @Test + public void shouldReturnSinkTask() { + assertEquals(ElasticsearchSinkTask.class, connector.taskClass()); + } + + @Test + public void shouldStartAndStop() { + connector.start(settings); + connector.stop(); + } + + @Test + public void testVersion() { + assertNotNull(connector.version()); + assertFalse(connector.version().equals("0.0.0.0")); + assertFalse(connector.version().equals("unknown")); + // Match semver with potentially a qualifier in the end + assertTrue(connector.version().matches("^(\\d+\\.){2}?(\\*|\\d+)(-.*)?$")); + } +} diff --git a/src/test/java/io/confluent/connect/elasticsearch/ElasticsearchSinkTaskTest.java b/src/test/java/io/confluent/connect/elasticsearch/ElasticsearchSinkTaskTest.java index 4df906591..994d71ec4 100644 --- a/src/test/java/io/confluent/connect/elasticsearch/ElasticsearchSinkTaskTest.java +++ b/src/test/java/io/confluent/connect/elasticsearch/ElasticsearchSinkTaskTest.java @@ -15,165 +15,340 @@ package io.confluent.connect.elasticsearch; +import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG; +import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.DROP_INVALID_MESSAGE_CONFIG; +import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.IGNORE_KEY_CONFIG; +import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.IGNORE_SCHEMA_CONFIG; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; -import com.carrotsearch.randomizedtesting.annotations.ThreadLeakScope; - -import org.apache.kafka.common.TopicPartition; +import io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.BehaviorOnNullValues; import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaBuilder; import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.errors.ConnectException; +import org.apache.kafka.connect.errors.DataException; +import org.apache.kafka.connect.sink.ErrantRecordReporter; import org.apache.kafka.connect.sink.SinkRecord; import org.apache.kafka.connect.sink.SinkTaskContext; -import org.elasticsearch.action.ActionFuture; -import org.elasticsearch.action.admin.indices.exists.indices.IndicesExistsRequest; -import org.elasticsearch.action.admin.indices.exists.indices.IndicesExistsResponse; -import org.elasticsearch.test.ESIntegTestCase; -import org.elasticsearch.test.InternalTestCluster; +import org.elasticsearch.action.DocWriteRequest; +import org.junit.Before; import org.junit.Test; -import java.util.Arrays; import java.util.HashMap; -import java.util.HashSet; import java.util.Map; -import java.util.Collection; -import java.util.ArrayList; import java.util.Collections; +public class ElasticsearchSinkTaskTest { + + protected static final String TOPIC = "topic"; -@ThreadLeakScope(ThreadLeakScope.Scope.NONE) -public class ElasticsearchSinkTaskTest extends ElasticsearchSinkTestBase { + protected ElasticsearchClient client; + private ElasticsearchSinkTask task; + private Map props; + private SinkTaskContext context; - private static final String TOPIC_IN_CAPS = "AnotherTopicInCaps"; - private static final int PARTITION_113 = 113; - private static final TopicPartition TOPIC_IN_CAPS_PARTITION = new TopicPartition(TOPIC_IN_CAPS, PARTITION_113); + private void setUpTask() { + task = new ElasticsearchSinkTask(); + task.initialize(context); + task.start(props, client); + } - private static final String UNSEEN_TOPIC = "UnseenTopic"; - private static final int PARTITION_114 = 114; - private static final TopicPartition UNSEEN_TOPIC_PARTITION = new TopicPartition(UNSEEN_TOPIC, PARTITION_114); + @Before + public void setUp() { + props = ElasticsearchSinkConnectorConfigTest.addNecessaryProps(new HashMap<>()); + props.put(IGNORE_KEY_CONFIG, "true"); + client = mock(ElasticsearchClient.class); + context = mock(SinkTaskContext.class); - private Map createProps() { - Map props = new HashMap<>(); - props.put(ElasticsearchSinkConnectorConfig.TYPE_NAME_CONFIG, TYPE); - props.put(ElasticsearchSinkConnectorConfig.CONNECTION_URL_CONFIG, "localhost"); - props.put(ElasticsearchSinkConnectorConfig.IGNORE_KEY_CONFIG, "true"); - props.put(ElasticsearchSinkConnectorConfig.READ_TIMEOUT_MS_CONFIG, "3000"); - return props; + setUpTask(); } @Test - public void testPutAndFlush() throws Exception { - InternalTestCluster cluster = ESIntegTestCase.internalCluster(); - cluster.ensureAtLeastNumDataNodes(3); - Map props = createProps(); + public void testPutSkipNullRecords() { + props.put(BEHAVIOR_ON_NULL_VALUES_CONFIG, BehaviorOnNullValues.IGNORE.name()); + setUpTask(); + + // skip null + SinkRecord nullRecord = record(true, true, 0); + task.put(Collections.singletonList(nullRecord)); + verify(client, never()).index(eq(nullRecord), any(DocWriteRequest.class)); + + // don't skip non-null + SinkRecord notNullRecord = record(true, false,1); + task.put(Collections.singletonList(notNullRecord)); + verify(client, times(1)).index(eq(notNullRecord), any(DocWriteRequest.class)); + } - ElasticsearchSinkTask task = new ElasticsearchSinkTask(); - task.initialize(mock(SinkTaskContext.class)); - task.start(props, client); - task.open(new HashSet<>(Arrays.asList(TOPIC_PARTITION, TOPIC_PARTITION2, TOPIC_PARTITION3))); + @Test + public void testReportNullRecords() { + ErrantRecordReporter mockReporter = mock(ErrantRecordReporter.class); + when(context.errantRecordReporter()).thenReturn(mockReporter); + + props.put(BEHAVIOR_ON_NULL_VALUES_CONFIG, BehaviorOnNullValues.IGNORE.name()); + setUpTask(); + + // report null + SinkRecord nullRecord = record(true, true, 0); + task.put(Collections.singletonList(nullRecord)); + verify(client, never()).index(eq(nullRecord), any(DocWriteRequest.class)); + verify(mockReporter, times(1)).report(eq(nullRecord), any(ConnectException.class)); + + // don't report + SinkRecord notNullRecord = record(true, false,1); + task.put(Collections.singletonList(notNullRecord)); + verify(client, times(1)).index(eq(notNullRecord), any(DocWriteRequest.class)); + verify(mockReporter, never()).report(eq(notNullRecord), any(ConnectException.class)); + } - String key = "key"; - Schema schema = createSchema(); - Struct record = createRecord(schema); + @Test(expected = DataException.class) + public void testPutFailNullRecords() { + props.put(BEHAVIOR_ON_NULL_VALUES_CONFIG, BehaviorOnNullValues.FAIL.name()); + setUpTask(); - Collection records = new ArrayList<>(); - SinkRecord sinkRecord = new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, schema, record, 0); - records.add(sinkRecord); + // fail null + SinkRecord nullRecord = record(true, true, 0); + task.put(Collections.singletonList(nullRecord)); + } - sinkRecord = new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, schema, record, 1); - records.add(sinkRecord); + @Test + public void testCreateIndex() { + task.put(Collections.singletonList(record())); + verify(client, times(1)).createIndex(eq(TOPIC)); + } - task.put(records); - task.flush(null); + @Test + public void testCreateUpperCaseIndex() { + task.put(Collections.singletonList(record())); + verify(client, times(1)).createIndex(eq(TOPIC.toLowerCase())); + } + + @Test + public void testDoNotCreateCachedIndex() { + task.put(Collections.singletonList(record())); + verify(client, times(1)).createIndex(eq(TOPIC)); - refresh(); + task.put(Collections.singletonList(record())); + verify(client, times(1)).createIndex(eq(TOPIC)); + } - verifySearchResults(records, true, false); + @Test + public void testIgnoreSchema() { + props.put(IGNORE_SCHEMA_CONFIG, "true"); + setUpTask(); + + SinkRecord record = record(); + task.put(Collections.singletonList(record)); + verify(client, never()).hasMapping(eq(TOPIC)); + verify(client, never()).createMapping(eq(TOPIC), eq(record.valueSchema())); } @Test - public void testCreateAndWriteToIndexForTopicWithUppercaseCharacters() { - // We should as well test that writing a record with a previously un seen record will create - // an index following the required elasticsearch requirements of lowercasing. - InternalTestCluster cluster = ESIntegTestCase.internalCluster(); - cluster.ensureAtLeastNumDataNodes(3); - Map props = createProps(); + public void testCheckMapping() { + when(client.hasMapping(TOPIC)).thenReturn(true); - ElasticsearchSinkTask task = new ElasticsearchSinkTask(); - task.initialize(mock(SinkTaskContext.class)); + SinkRecord record = record(); + task.put(Collections.singletonList(record)); + verify(client, times(1)).hasMapping(eq(TOPIC)); + verify(client, never()).createMapping(eq(TOPIC), eq(record.valueSchema())); + } + + @Test + public void testAddMapping() { + SinkRecord record = record(); + task.put(Collections.singletonList(record)); + verify(client, times(1)).hasMapping(eq(TOPIC)); + verify(client, times(1)).createMapping(eq(TOPIC), eq(record.valueSchema())); + } - String key = "key"; - Schema schema = createSchema(); - Struct record = createRecord(schema); + @Test + public void testDoNotAddCachedMapping() { + SinkRecord record = record(); + task.put(Collections.singletonList(record)); + verify(client, times(1)).hasMapping(eq(TOPIC)); + verify(client, times(1)).createMapping(eq(TOPIC), eq(record.valueSchema())); + + task.put(Collections.singletonList(record)); + verify(client, times(1)).hasMapping(eq(TOPIC)); + verify(client, times(1)).createMapping(eq(TOPIC), eq(record.valueSchema())); + } - SinkRecord sinkRecord = new SinkRecord(TOPIC_IN_CAPS, - PARTITION_113, - Schema.STRING_SCHEMA, - key, - schema, - record, - 0 ); + @Test + public void testPut() { + SinkRecord record = record(); + task.put(Collections.singletonList(record)); + verify(client, times(1)).index(eq(record), any()); + } - try { - task.start(props, client); - task.open(new HashSet<>(Collections.singletonList(TOPIC_IN_CAPS_PARTITION))); - task.put(Collections.singleton(sinkRecord)); - } catch (Exception ex) { - fail("A topic name not in lowercase can not be used as index name in Elasticsearch"); - } finally { - task.stop(); - } + @Test + public void testPutSkipInvalidRecord() { + props.put(DROP_INVALID_MESSAGE_CONFIG, "true"); + props.put(IGNORE_KEY_CONFIG, "false"); + setUpTask(); + + // skip invalid + SinkRecord invalidRecord = record(true, 0); + task.put(Collections.singletonList(invalidRecord)); + verify(client, never()).index(eq(invalidRecord), any(DocWriteRequest.class)); + + // don't skip valid + SinkRecord validRecord = record(false, 1); + task.put(Collections.singletonList(validRecord)); + verify(client, times(1)).index(eq(validRecord), any(DocWriteRequest.class)); } @Test - public void testCreateAndWriteToIndexNotCreatedAtStartTime() { - InternalTestCluster cluster = ESIntegTestCase.internalCluster(); - cluster.ensureAtLeastNumDataNodes(3); - Map props = createProps(); + public void testPutReportInvalidRecord() { + ErrantRecordReporter mockReporter = mock(ErrantRecordReporter.class); + when(context.errantRecordReporter()).thenReturn(mockReporter); + + props.put(DROP_INVALID_MESSAGE_CONFIG, "true"); + props.put(IGNORE_KEY_CONFIG, "false"); + setUpTask(); + + // report invalid + SinkRecord invalidRecord = record(true, 0); + task.put(Collections.singletonList(invalidRecord)); + verify(client, never()).index(eq(invalidRecord), any(DocWriteRequest.class)); + verify(mockReporter, times(1)).report(eq(invalidRecord), any(DataException.class)); + + // don't report valid + SinkRecord validRecord = record(false, 1); + task.put(Collections.singletonList(validRecord)); + verify(client, times(1)).index(eq(validRecord), any(DocWriteRequest.class)); + verify(mockReporter, never()).report(eq(validRecord), any(DataException.class)); + } - props.put(ElasticsearchSinkConnectorConfig.CREATE_INDICES_AT_START_CONFIG, "false"); + @Test(expected = DataException.class) + public void testPutFailsOnInvalidRecord() { + props.put(DROP_INVALID_MESSAGE_CONFIG, "false"); + props.put(IGNORE_KEY_CONFIG, "false"); + setUpTask(); - ElasticsearchSinkTask task = new ElasticsearchSinkTask(); - task.initialize(mock(SinkTaskContext.class)); + SinkRecord invalidRecord = record(); + task.put(Collections.singletonList(invalidRecord)); + } - String key = "key"; - Schema schema = createSchema(); - Struct record = createRecord(schema); + @Test + public void testFlush() { + setUpTask(); + task.flush(null); + verify(client, times(1)).flush(); + } - SinkRecord sinkRecord = new SinkRecord(UNSEEN_TOPIC, - PARTITION_114, - Schema.STRING_SCHEMA, - key, - schema, - record, - 0 ); + @Test + public void testFlushDoesNotThrow() { + setUpTask(); + doThrow(new IllegalStateException("already closed")).when(client).flush(); - task.start(props, client); - task.open(new HashSet<>(Collections.singletonList(TOPIC_IN_CAPS_PARTITION))); - task.put(Collections.singleton(sinkRecord)); + // should not throw + task.flush(null); + verify(client, times(1)).flush(); + } + + @Test + public void testStartAndStop() { + task = new ElasticsearchSinkTask(); + task.initialize(context); + task.start(props); task.stop(); + } - assertTrue(UNSEEN_TOPIC + " index created without errors ", - verifyIndexExist(cluster, UNSEEN_TOPIC.toLowerCase())); + @Test + public void testVersion() { + setUpTask(); + assertNotNull(task.version()); + assertFalse(task.version().equals("0.0.0.0")); + // Match semver with potentially a qualifier in the end + assertTrue(task.version().matches("^(\\d+\\.){2}?(\\*|\\d+)(-.*)?$")); + } + @Test + public void testConvertTopicToIndexName() { + setUpTask(); + + String upperCaseTopic = "UPPERCASE"; + SinkRecord record = record(upperCaseTopic, true, false, 0); + task.put(Collections.singletonList(record)); + verify(client, times(1)).createIndex(eq(upperCaseTopic.toLowerCase())); + + String tooLongTopic = String.format("%0256d", 1); + record = record(tooLongTopic, true, false, 0); + task.put(Collections.singletonList(record)); + verify(client, times(1)).createIndex(eq(tooLongTopic.substring(0, 255))); + + String startsWithDash = "-dash"; + record = record(startsWithDash, true, false, 0); + task.put(Collections.singletonList(record)); + verify(client, times(1)).createIndex(eq("dash")); + + String startsWithUnderscore = "_underscore"; + record = record(startsWithUnderscore, true, false, 0); + task.put(Collections.singletonList(record)); + verify(client, times(1)).createIndex(eq("underscore")); + + String dot = "."; + record = record(dot, true, false, 0); + task.put(Collections.singletonList(record)); + verify(client, times(1)).createIndex(eq("dot")); + + String dots = ".."; + record = record(dots, true, false, 0); + task.put(Collections.singletonList(record)); + verify(client, times(1)).createIndex(eq("dotdot")); } @Test - public void testStopThenFlushDoesNotThrow() { - ElasticsearchSinkTask task = new ElasticsearchSinkTask(); - task.initialize(mock(SinkTaskContext.class)); - task.start(createProps(), client); - task.stop(); - task.flush(new HashMap<>()); + public void testShouldNotThrowIfReporterDoesNotExist() { + when(context.errantRecordReporter()) + .thenThrow(new NoSuchMethodError("what are you doing")) + .thenThrow(new NoClassDefFoundError("i no exist")); + + // call start twice for both exceptions + setUpTask(); + setUpTask(); + } + + private SinkRecord record() { + return record(true, false,0); } - private boolean verifyIndexExist(InternalTestCluster cluster, String ... indices) { - ActionFuture action = cluster - .client() - .admin() - .indices() - .exists(new IndicesExistsRequest(indices)); + private SinkRecord record(boolean nullKey, long offset) { + return record(nullKey, false, offset); + } - return action.actionGet().isExists(); + private SinkRecord record(boolean nullKey, boolean nullValue, long offset) { + return record(TOPIC, nullKey, nullValue, offset); } + + private SinkRecord record(String topic, boolean nullKey, boolean nullValue, long offset) { + Schema schema = SchemaBuilder.struct().name("struct") + .field("user", Schema.STRING_SCHEMA) + .field("message", Schema.STRING_SCHEMA) + .build(); + + Struct struct = new Struct(schema); + struct.put("user", "Liquan"); + struct.put("message", "trying out Elastic Search."); + + return new SinkRecord( + topic, + 1, + Schema.STRING_SCHEMA, + nullKey ? null : "key", + schema, + nullValue ? null : struct, + offset + ); +} } diff --git a/src/test/java/io/confluent/connect/elasticsearch/ElasticsearchSinkTestBase.java b/src/test/java/io/confluent/connect/elasticsearch/ElasticsearchSinkTestBase.java deleted file mode 100644 index 536ef0b9b..000000000 --- a/src/test/java/io/confluent/connect/elasticsearch/ElasticsearchSinkTestBase.java +++ /dev/null @@ -1,166 +0,0 @@ -/* - * 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.connect.elasticsearch; - -import com.google.gson.JsonArray; -import com.google.gson.JsonObject; - -import io.confluent.connect.elasticsearch.jest.JestElasticsearchClient; -import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.connect.data.Schema; -import org.apache.kafka.connect.data.SchemaBuilder; -import org.apache.kafka.connect.data.Struct; -import org.apache.kafka.connect.sink.SinkRecord; -import org.elasticsearch.cluster.metadata.IndexMetaData; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.node.Node; -import org.elasticsearch.test.ESIntegTestCase; -import org.junit.After; -import org.junit.Before; - -import java.io.IOException; -import java.util.Collection; -import java.util.HashMap; -import java.util.Map; - -import static io.confluent.connect.elasticsearch.DataConverter.BehaviorOnNullValues; - -public class ElasticsearchSinkTestBase extends ESIntegTestCase { - - protected static final String TYPE = "kafka-connect"; - - protected static final String TOPIC = "topic"; - protected static final int PARTITION = 12; - protected static final int PARTITION2 = 13; - protected static final int PARTITION3 = 14; - protected static final TopicPartition TOPIC_PARTITION = new TopicPartition(TOPIC, PARTITION); - protected static final TopicPartition TOPIC_PARTITION2 = new TopicPartition(TOPIC, PARTITION2); - protected static final TopicPartition TOPIC_PARTITION3 = new TopicPartition(TOPIC, PARTITION3); - - protected ElasticsearchClient client; - private DataConverter converter; - - @Before - public void setUp() throws Exception { - super.setUp(); - client = new JestElasticsearchClient("http://localhost:" + getPort()); - converter = new DataConverter(true, BehaviorOnNullValues.IGNORE); - } - - @After - public void tearDown() throws Exception { - super.tearDown(); - if (client != null) { - client.close(); - } - client = null; - } - - protected int getPort() { - assertTrue("There should be at least 1 HTTP endpoint exposed in the test cluster", - cluster().httpAddresses().length > 0); - return cluster().httpAddresses()[0].getPort(); - } - - protected Struct createRecord(Schema schema) { - Struct struct = new Struct(schema); - struct.put("user", "Liquan"); - struct.put("message", "trying out Elastic Search."); - return struct; - } - - protected Schema createSchema() { - return SchemaBuilder.struct().name("record") - .field("user", Schema.STRING_SCHEMA) - .field("message", Schema.STRING_SCHEMA) - .build(); - } - - protected Schema createOtherSchema() { - return SchemaBuilder.struct().name("record") - .field("user", Schema.INT32_SCHEMA) - .build(); - } - - protected Struct createOtherRecord(Schema schema) { - Struct struct = new Struct(schema); - struct.put("user", 10); - return struct; - } - - protected void verifySearchResults(Collection records, boolean ignoreKey, boolean ignoreSchema) throws IOException { - verifySearchResults(records, TOPIC, ignoreKey, ignoreSchema); - } - - protected void verifySearchResults(Collection records, String index, boolean ignoreKey, boolean ignoreSchema) throws IOException { - final JsonObject result = client.search("", index, null); - - final JsonArray rawHits = result.getAsJsonObject("hits").getAsJsonArray("hits"); - - assertEquals(records.size(), rawHits.size()); - - Map hits = new HashMap<>(); - for (int i = 0; i < rawHits.size(); ++i) { - final JsonObject hitData = rawHits.get(i).getAsJsonObject(); - final String id = hitData.get("_id").getAsString(); - final String source = hitData.get("_source").getAsJsonObject().toString(); - hits.put(id, source); - } - - for (Object record : records) { - if (record instanceof SinkRecord) { - IndexableRecord indexableRecord = converter.convertRecord((SinkRecord) record, index, TYPE, ignoreKey, ignoreSchema); - assertEquals(indexableRecord.payload, hits.get(indexableRecord.key.id)); - } else { - assertEquals(record, hits.get("key")); - } - } - } - - /* For ES 2.x */ - @Override - protected Settings nodeSettings(int nodeOrdinal) { - return Settings.settingsBuilder() - .put(super.nodeSettings(nodeOrdinal)) - .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1) - .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1) - .put(Node.HTTP_ENABLED, true) - .build(); - } - - /* For ES 5.x (requires Java 8) */ - /* - @Override - protected Settings nodeSettings(int nodeOrdinal) { - int randomPort = randomIntBetween(49152, 65525); - return Settings.builder() - .put(super.nodeSettings(nodeOrdinal)) - .put(NetworkModule.HTTP_ENABLED.getKey(), true) - .put(HttpTransportSettings.SETTING_HTTP_PORT.getKey(), randomPort) - .put("network.host", "127.0.0.1") - .build(); - } - - @Override - protected Collection> nodePlugins() { - System.setProperty("es.set.netty.runtime.available.processors", "false"); - Collection> al = new ArrayList>(); - al.add(Netty4Plugin.class); - return al; - } - */ - -} diff --git a/src/test/java/io/confluent/connect/elasticsearch/ElasticsearchWriterTest.java b/src/test/java/io/confluent/connect/elasticsearch/ElasticsearchWriterTest.java deleted file mode 100644 index 488427df9..000000000 --- a/src/test/java/io/confluent/connect/elasticsearch/ElasticsearchWriterTest.java +++ /dev/null @@ -1,564 +0,0 @@ -/* - * 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.connect.elasticsearch; - -import com.carrotsearch.randomizedtesting.annotations.ThreadLeakScope; -import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.kafka.connect.data.Decimal; -import org.apache.kafka.connect.data.Schema; -import org.apache.kafka.connect.data.SchemaBuilder; -import org.apache.kafka.connect.data.Struct; -import org.apache.kafka.connect.errors.ConnectException; -import org.apache.kafka.connect.errors.DataException; -import org.apache.kafka.connect.sink.SinkRecord; -import org.hamcrest.MatcherAssert; -import org.junit.Before; -import org.junit.Test; - -import java.math.BigDecimal; -import java.math.BigInteger; -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; - -import static io.confluent.connect.elasticsearch.DataConverter.BehaviorOnNullValues; -import static org.hamcrest.Matchers.containsString; - -@ThreadLeakScope(ThreadLeakScope.Scope.NONE) -public class ElasticsearchWriterTest extends ElasticsearchSinkTestBase { - - private final String key = "key"; - private final Schema schema = createSchema(); - private final Struct record = createRecord(schema); - private final Schema otherSchema = createOtherSchema(); - private final Struct otherRecord = createOtherRecord(otherSchema); - - private boolean ignoreKey; - private boolean ignoreSchema; - - @Before - public void setUp() throws Exception { - ignoreKey = false; - ignoreSchema = false; - - super.setUp(); - } - - @Test - public void testWriter() throws Exception { - Collection records = prepareData(2); - ElasticsearchWriter writer = initWriter(client); - writeDataAndRefresh(writer, records); - - Collection expected = Collections.singletonList( - new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, schema, record, 1) - ); - verifySearchResults(expected); - } - - @Test - public void testWriterIgnoreKey() throws Exception { - ignoreKey = true; - - Collection records = prepareData(2); - ElasticsearchWriter writer = initWriter(client); - writeDataAndRefresh(writer, records); - verifySearchResults(records); - } - - @Test - public void testWriterIgnoreSchema() throws Exception { - ignoreKey = true; - ignoreSchema = true; - - Collection records = prepareData(2); - ElasticsearchWriter writer = initWriter(client); - writeDataAndRefresh(writer, records); - verifySearchResults(records); - } - - @Test - public void testTopicIndexOverride() throws Exception { - ignoreKey = true; - ignoreSchema = true; - - String indexOverride = "index"; - - Collection records = prepareData(2); - ElasticsearchWriter writer = initWriter( - client, - Collections.emptySet(), - Collections.emptySet(), - Collections.singletonMap(TOPIC, indexOverride), - false, - BehaviorOnNullValues.IGNORE); - writeDataAndRefresh(writer, records); - verifySearchResults(records, indexOverride); - } - - @Test - public void testIncompatible() throws Exception { - ignoreKey = true; - - Collection records = new ArrayList<>(); - SinkRecord sinkRecord = - new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, otherSchema, otherRecord, 0); - records.add(sinkRecord); - - ElasticsearchWriter writer = initWriter(client); - - writer.write(records); - Thread.sleep(5000); - records.clear(); - - sinkRecord = - new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, schema, record, 1); - records.add(sinkRecord); - writer.write(records); - - try { - writer.flush(); - fail("should fail because of mapper_parsing_exception"); - } catch (ConnectException e) { - // expected - } - } - - @Test - public void testCompatible() throws Exception { - ignoreKey = true; - - Collection records = new ArrayList<>(); - Collection expected = new ArrayList<>(); - SinkRecord sinkRecord = - new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, schema, record, 0); - records.add(sinkRecord); - expected.add(sinkRecord); - sinkRecord = - new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, schema, record, 1); - records.add(sinkRecord); - expected.add(sinkRecord); - - ElasticsearchWriter writer = initWriter(client); - - writer.write(records); - records.clear(); - - sinkRecord = - new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, otherSchema, otherRecord, 2); - records.add(sinkRecord); - expected.add(sinkRecord); - - sinkRecord = - new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, otherSchema, otherRecord, 3); - records.add(sinkRecord); - expected.add(sinkRecord); - - writeDataAndRefresh(writer, records); - verifySearchResults(expected); - } - - @Test - public void testSafeRedeliveryRegularKey() throws Exception { - Struct value0 = new Struct(schema); - value0.put("user", "foo"); - value0.put("message", "hi"); - SinkRecord sinkRecord0 = - new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, schema, value0, 0); - - Struct value1 = new Struct(schema); - value1.put("user", "foo"); - value1.put("message", "bye"); - SinkRecord sinkRecord1 = - new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, schema, value1, 1); - - ElasticsearchWriter writer = initWriter(client); - writer.write(Arrays.asList(sinkRecord0, sinkRecord1)); - writer.flush(); - - // write the record with earlier offset again - writeDataAndRefresh(writer, Collections.singleton(sinkRecord0)); - - // last write should have been ignored due to version conflict - verifySearchResults(Collections.singleton(sinkRecord1)); - } - - @Test - public void testSafeRedeliveryOffsetInKey() throws Exception { - ignoreKey = true; - - Struct value0 = new Struct(schema); - value0.put("user", "foo"); - value0.put("message", "hi"); - SinkRecord sinkRecord0 = - new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, schema, value0, 0); - - Struct value1 = new Struct(schema); - value1.put("user", "foo"); - value1.put("message", "bye"); - SinkRecord sinkRecord1 = - new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, schema, value1, 1); - - List records = Arrays.asList(sinkRecord0, sinkRecord1); - - ElasticsearchWriter writer = initWriter(client); - writer.write(records); - writer.flush(); - - // write them again - writeDataAndRefresh(writer, records); - - // last write should have been ignored due to version conflict - verifySearchResults(records); - } - - @Test - public void testMap() throws Exception { - Schema structSchema = SchemaBuilder.struct().name("struct") - .field("map", SchemaBuilder.map(Schema.INT32_SCHEMA, Schema.STRING_SCHEMA).build()) - .build(); - - Map map = new HashMap<>(); - map.put(1, "One"); - map.put(2, "Two"); - - Struct struct = new Struct(structSchema); - struct.put("map", map); - - Collection records = new ArrayList<>(); - SinkRecord sinkRecord = - new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, structSchema, struct, 0); - records.add(sinkRecord); - - ElasticsearchWriter writer = initWriter(client); - writeDataAndRefresh(writer, records); - verifySearchResults(records); - } - - @Test - public void testStringKeyedMap() throws Exception { - Schema mapSchema = SchemaBuilder.map(Schema.STRING_SCHEMA, Schema.INT32_SCHEMA).build(); - - Map map = new HashMap<>(); - map.put("One", 1); - map.put("Two", 2); - - SinkRecord sinkRecord = - new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, mapSchema, map, 0); - - ElasticsearchWriter writer = initWriter(client); - writeDataAndRefresh(writer, Collections.singletonList(sinkRecord)); - - Collection expectedRecords = - Collections.singletonList(new ObjectMapper().writeValueAsString(map)); - verifySearchResults(expectedRecords, TOPIC); - } - - @Test - public void testDecimal() throws Exception { - int scale = 2; - byte[] bytes = ByteBuffer.allocate(4).putInt(2).array(); - BigDecimal decimal = new BigDecimal(new BigInteger(bytes), scale); - - Schema structSchema = SchemaBuilder.struct().name("struct") - .field("decimal", Decimal.schema(scale)) - .build(); - - Struct struct = new Struct(structSchema); - struct.put("decimal", decimal); - - Collection records = new ArrayList<>(); - SinkRecord sinkRecord = - new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, structSchema, struct, 0); - records.add(sinkRecord); - - ElasticsearchWriter writer = initWriter(client); - writeDataAndRefresh(writer, records); - verifySearchResults(records); - } - - @Test - public void testBytes() throws Exception { - Schema structSchema = SchemaBuilder.struct().name("struct") - .field("bytes", SchemaBuilder.BYTES_SCHEMA) - .build(); - - Struct struct = new Struct(structSchema); - struct.put("bytes", new byte[]{42}); - - Collection records = new ArrayList<>(); - SinkRecord sinkRecord = - new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, structSchema, struct, 0); - records.add(sinkRecord); - - ElasticsearchWriter writer = initWriter(client); - writeDataAndRefresh(writer, records); - verifySearchResults(records); - } - - @Test - public void testIgnoreNullValue() throws Exception { - Collection records = new ArrayList<>(); - SinkRecord sinkRecord = - new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, schema, null, 0); - records.add(sinkRecord); - - ElasticsearchWriter writer = initWriter(client, BehaviorOnNullValues.IGNORE); - writeDataAndRefresh(writer, records); - // Send an empty list of records to the verify method, since the empty record should have been - // skipped - verifySearchResults(new ArrayList()); - } - - @Test - public void testDeleteOnNullValue() throws Exception { - String key1 = "key1"; - String key2 = "key2"; - - ElasticsearchWriter writer = initWriter(client, BehaviorOnNullValues.DELETE); - - Collection records = new ArrayList<>(); - - // First, write a couple of actual (non-null-valued) records - SinkRecord insertRecord1 = - new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key1, schema, record, 0); - records.add(insertRecord1); - SinkRecord insertRecord2 = - new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key2, otherSchema, otherRecord, 1); - records.add(insertRecord2); - // Can't call writeDataAndRefresh(writer, records) since it stops the writer - writer.write(records); - writer.flush(); - refresh(); - // Make sure the record made it there successfully - verifySearchResults(records); - - // Then, write a record with the same key as the first inserted record but a null value - SinkRecord deleteRecord = - new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key1, schema, null, 2); - - // Don't want to resend the first couple of records - records.clear(); - records.add(deleteRecord); - writeDataAndRefresh(writer, records); - - // The only remaining record should be the second inserted record - records.clear(); - records.add(insertRecord2); - verifySearchResults(records); - } - - @Test - public void testIneffectiveDelete() throws Exception { - // Just a sanity check to make sure things don't blow up if an attempt is made to delete a - // record that doesn't exist in the first place - - Collection records = new ArrayList<>(); - SinkRecord sinkRecord = - new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, schema, null, 0); - records.add(sinkRecord); - - ElasticsearchWriter writer = initWriter(client, BehaviorOnNullValues.DELETE); - writeDataAndRefresh(writer, records); - verifySearchResults(new ArrayList()); - } - - @Test - public void testDeleteWithNullKey() throws Exception { - Collection records = new ArrayList<>(); - SinkRecord sinkRecord = - new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, null, schema, null, 0); - records.add(sinkRecord); - - ElasticsearchWriter writer = initWriter(client, BehaviorOnNullValues.DELETE); - writeDataAndRefresh(writer, records); - verifySearchResults(new ArrayList()); - } - - @Test - public void testFailOnNullValue() throws Exception { - Collection records = new ArrayList<>(); - SinkRecord sinkRecord = - new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, schema, null, 0); - records.add(sinkRecord); - - ElasticsearchWriter writer = initWriter(client, BehaviorOnNullValues.FAIL); - try { - writeDataAndRefresh(writer, records); - fail("should fail because of behavior.on.null.values=fail"); - } catch (DataException e) { - // expected - } - } - - @Test - public void testInvalidRecordException() throws Exception { - ignoreSchema = true; - - Collection records = new ArrayList<>(); - - SinkRecord sinkRecord = - new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, null, null, new byte[]{42}, 0); - records.add(sinkRecord); - - final ElasticsearchWriter strictWriter = initWriter(client); - - Exception e = assertThrows(ConnectException.class, () -> { - strictWriter.write(records); - }); - assertEquals("Key is used as document id and can not be null.", e.getMessage()); - } - - @Test - public void testDropInvalidRecord() throws Exception { - ignoreSchema = true; - Collection inputRecords = new ArrayList<>(); - Collection outputRecords = new ArrayList<>(); - - Schema structSchema = SchemaBuilder.struct().name("struct") - .field("bytes", SchemaBuilder.BYTES_SCHEMA) - .build(); - - Struct struct = new Struct(structSchema); - struct.put("bytes", new byte[]{42}); - - - SinkRecord invalidRecord = new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, null, structSchema, struct, 0); - SinkRecord validRecord = new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, structSchema, struct, 1); - - inputRecords.add(validRecord); - inputRecords.add(invalidRecord); - - outputRecords.add(validRecord); - - final ElasticsearchWriter nonStrictWriter = initWriter(client, true); - - writeDataAndRefresh(nonStrictWriter, inputRecords); - verifySearchResults(outputRecords, ignoreKey, ignoreSchema); - } - - @Test - public void testDropInvalidRecordThrowsOnOtherErrors() throws Exception { - ignoreSchema = true; - Collection inputRecords = new ArrayList<>(); - - Schema structSchema = SchemaBuilder.struct().name("struct") - .field("bytes", SchemaBuilder.BYTES_SCHEMA) - .build(); - - Struct struct = new Struct(structSchema); - struct.put("bytes", new byte[]{42}); - - SinkRecord validRecord = new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, structSchema, struct, 1); - - inputRecords.add(validRecord); - - final ElasticsearchWriter nonStrictWriter = initWriter(client, true); - // stop the bulk processor - nonStrictWriter.stop(); - - // try to write on a stopped writer, should throw - ConnectException e = assertThrows(ConnectException.class, - () -> nonStrictWriter.write(inputRecords)); - MatcherAssert.assertThat(e.getMessage(), containsString("Stopping")); - } - - private Collection prepareData(int numRecords) { - Collection records = new ArrayList<>(); - for (int i = 0; i < numRecords; ++i) { - SinkRecord sinkRecord = - new SinkRecord(TOPIC, PARTITION, Schema.STRING_SCHEMA, key, schema, record, i); - records.add(sinkRecord); - } - return records; - } - - private ElasticsearchWriter initWriter(ElasticsearchClient client) { - return initWriter(client, false, BehaviorOnNullValues.IGNORE); - } - - private ElasticsearchWriter initWriter(ElasticsearchClient client, boolean dropInvalidMessage) { - return initWriter(client, dropInvalidMessage, BehaviorOnNullValues.IGNORE); - } - - private ElasticsearchWriter initWriter(ElasticsearchClient client, BehaviorOnNullValues behavior) { - return initWriter(client, false, behavior); - } - - private ElasticsearchWriter initWriter( - ElasticsearchClient client, - boolean dropInvalidMessage, - BehaviorOnNullValues behavior) { - return initWriter( - client, - Collections.emptySet(), - Collections.emptySet(), - Collections.emptyMap(), - dropInvalidMessage, - behavior - ); - } - - private ElasticsearchWriter initWriter( - ElasticsearchClient client, - Set ignoreKeyTopics, - Set ignoreSchemaTopics, - Map topicToIndexMap, - boolean dropInvalidMessage, - BehaviorOnNullValues behavior - ) { - ElasticsearchWriter writer = new ElasticsearchWriter.Builder(client) - .setType(TYPE) - .setIgnoreKey(ignoreKey, ignoreKeyTopics) - .setIgnoreSchema(ignoreSchema, ignoreSchemaTopics) - .setTopicToIndexMap(topicToIndexMap) - .setFlushTimoutMs(10000) - .setMaxBufferedRecords(10000) - .setMaxInFlightRequests(1) - .setBatchSize(2) - .setLingerMs(1000) - .setRetryBackoffMs(1000) - .setMaxRetry(3) - .setDropInvalidMessage(dropInvalidMessage) - .setBehaviorOnNullValues(behavior) - .build(); - writer.start(); - writer.createIndicesForTopics(Collections.singleton(TOPIC)); - return writer; - } - - private void writeDataAndRefresh(ElasticsearchWriter writer, Collection records) - throws Exception { - writer.write(records); - writer.flush(); - writer.stop(); - refresh(); - } - - private void verifySearchResults(Collection records) throws Exception { - verifySearchResults(records, ignoreKey, ignoreSchema); - } - - private void verifySearchResults(Collection records, String index) throws Exception { - verifySearchResults(records, index, ignoreKey, ignoreSchema); - } -} diff --git a/src/test/java/io/confluent/connect/elasticsearch/MappingTest.java b/src/test/java/io/confluent/connect/elasticsearch/MappingTest.java index b272c2d06..c3fe6ea97 100644 --- a/src/test/java/io/confluent/connect/elasticsearch/MappingTest.java +++ b/src/test/java/io/confluent/connect/elasticsearch/MappingTest.java @@ -15,12 +15,11 @@ package io.confluent.connect.elasticsearch; -import com.fasterxml.jackson.databind.node.NumericNode; -import com.fasterxml.jackson.databind.node.ObjectNode; -import com.fasterxml.jackson.databind.node.TextNode; -import com.fasterxml.jackson.databind.JsonNode; import com.google.gson.JsonObject; - +import com.google.gson.JsonParser; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.util.HashMap; import org.apache.kafka.connect.data.Date; import org.apache.kafka.connect.data.Decimal; import org.apache.kafka.connect.data.Field; @@ -28,125 +27,109 @@ import org.apache.kafka.connect.data.SchemaBuilder; import org.apache.kafka.connect.data.Time; import org.apache.kafka.connect.data.Timestamp; -import org.elasticsearch.test.ESIntegTestCase; -import org.elasticsearch.test.InternalTestCluster; +import org.apache.kafka.connect.errors.DataException; +import org.elasticsearch.common.xcontent.XContentBuilder; import org.junit.Test; -import static io.confluent.connect.elasticsearch.DataConverter.BehaviorOnNullValues; -import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConstants.KEYWORD_TYPE; -import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConstants.TEXT_TYPE; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; +import static io.confluent.connect.elasticsearch.Mapping.KEYWORD_TYPE; +import static io.confluent.connect.elasticsearch.Mapping.KEY_FIELD; +import static io.confluent.connect.elasticsearch.Mapping.TEXT_TYPE; +import static io.confluent.connect.elasticsearch.Mapping.VALUE_FIELD; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; @SuppressWarnings("deprecation") -public class MappingTest extends ElasticsearchSinkTestBase { - - private static final String INDEX = "kafka-connect"; - private static final String TYPE = "kafka-connect-type"; +public class MappingTest { - @Test - @SuppressWarnings("unchecked") - public void testMapping() throws Exception { - InternalTestCluster cluster = ESIntegTestCase.internalCluster(); - cluster.ensureAtLeastNumDataNodes(1); - - createIndex(INDEX); - Schema schema = createSchema(); - Mapping.createMapping(client, INDEX, TYPE, schema); - - JsonObject mapping = Mapping.getMapping(client, INDEX, TYPE); - assertNotNull(mapping); - verifyMapping(schema, mapping); + @Test(expected = DataException.class) + public void testBuildMappingWithNullSchema() { + XContentBuilder builder = Mapping.buildMapping(null); } @Test - @SuppressWarnings("unchecked") - public void testStringMappingForES6() throws Exception { - ElasticsearchClient client = mock(ElasticsearchClient.class); - when(client.getVersion()).thenReturn(ElasticsearchClient.Version.ES_V6); - - Schema schema = SchemaBuilder.struct().name("textRecord") - .field("string", Schema.STRING_SCHEMA) - .build(); - ObjectNode mapping = (ObjectNode) Mapping.inferMapping(client, schema); - ObjectNode properties = mapping.with("properties"); - ObjectNode string = properties.with("string"); - TextNode stringType = (TextNode) string.get("type"); - ObjectNode fields = string.with("fields"); - ObjectNode keyword = fields.with("keyword"); - TextNode keywordType = (TextNode) keyword.get("type"); - NumericNode ignoreAbove = (NumericNode) keyword.get("ignore_above"); - - assertEquals(TEXT_TYPE, stringType.asText()); - assertEquals(KEYWORD_TYPE, keywordType.asText()); - assertEquals(256, ignoreAbove.asInt()); + public void testBuildMapping() throws IOException { + JsonObject result = runTest(createSchema()); + verifyMapping(createSchema(), result); } @Test - public void testInferMapping() throws Exception { - - Schema stringSchema = SchemaBuilder - .struct() + public void testBuildMappingForString() throws IOException { + Schema schema = SchemaBuilder.struct() .name("record") - .field("foo", SchemaBuilder.string().defaultValue("0").build()) + .field("string", Schema.STRING_SCHEMA) .build(); - JsonNode stringMapping = Mapping.inferMapping(client, stringSchema); - assertNull(stringMapping.get("properties").get("foo").get("null_value")); + JsonObject result = runTest(schema); + JsonObject string = result.getAsJsonObject("properties").getAsJsonObject("string"); + JsonObject keyword = string.getAsJsonObject("fields").getAsJsonObject("keyword"); + + assertEquals(TEXT_TYPE, string.get("type").getAsString()); + assertEquals(KEYWORD_TYPE, keyword.get("type").getAsString()); + assertEquals(256, keyword.get("ignore_above").getAsInt()); + } - Schema intSchema =SchemaBuilder + @Test + public void testBuildMappingSetsDefaultValue() throws IOException { + Schema schema = SchemaBuilder .struct() .name("record") - .field("foo", SchemaBuilder.int32().defaultValue(0).build()) + .field("boolean", SchemaBuilder.bool().defaultValue(true).build()) + .field("int8", SchemaBuilder.int8().defaultValue((byte) 1).build()) + .field("int16", SchemaBuilder.int16().defaultValue((short) 1).build()) + .field("int32", SchemaBuilder.int32().defaultValue(1).build()) + .field("int64", SchemaBuilder.int64().defaultValue((long) 1).build()) + .field("float32", SchemaBuilder.float32().defaultValue((float) 1).build()) + .field("float64", SchemaBuilder.float64().defaultValue((double) 1).build()) .build(); - JsonNode intMapping = Mapping.inferMapping(client, intSchema); - assertNotNull(intMapping.get("properties").get("foo").get("null_value")); - assertEquals(0, intMapping.get("properties").get("foo").get("null_value").asInt()); + JsonObject properties = runTest(schema).getAsJsonObject("properties"); + assertEquals(1, properties.getAsJsonObject("int8").get("null_value").getAsInt()); + assertEquals(1, properties.getAsJsonObject("int16").get("null_value").getAsInt()); + assertEquals(1, properties.getAsJsonObject("int32").get("null_value").getAsInt()); + assertEquals(1, properties.getAsJsonObject("int64").get("null_value").getAsInt()); + assertEquals(1, properties.getAsJsonObject("float32").get("null_value").getAsInt()); + assertEquals(1, properties.getAsJsonObject("float64").get("null_value").getAsInt()); + assertEquals(true, properties.getAsJsonObject("boolean").get("null_value").getAsBoolean()); } @Test - public void testInferMappingDefaultDate() { + public void testBuildMappingSetsDefaultValueForDate() throws IOException { java.util.Date expected = new java.util.Date(); - - Schema dateSchema = SchemaBuilder + Schema schema = SchemaBuilder .struct() .name("record") - .field("foo", Date.builder().defaultValue(expected).build()) + .field("date", Date.builder().defaultValue(expected).build()) .build(); - JsonNode dateMapping = Mapping.inferMapping(client, dateSchema); - assertNotNull(dateMapping.get("properties").get("foo").get("null_value")); + JsonObject result = runTest(schema); + assertEquals( expected.getTime(), - dateMapping.get("properties").get("foo").get("null_value").asLong() + result.getAsJsonObject("properties").getAsJsonObject("date").get("null_value").getAsLong() ); } - protected Schema createSchema() { - Schema structSchema = createInnerSchema(); - return SchemaBuilder.struct().name("record") - .field("boolean", Schema.BOOLEAN_SCHEMA) - .field("bytes", Schema.BYTES_SCHEMA) - .field("int8", Schema.INT8_SCHEMA) - .field("int16", Schema.INT16_SCHEMA) - .field("int32", Schema.INT32_SCHEMA) - .field("int64", Schema.INT64_SCHEMA) - .field("float32", Schema.FLOAT32_SCHEMA) - .field("float64", Schema.FLOAT64_SCHEMA) - .field("string", Schema.STRING_SCHEMA) - .field("array", SchemaBuilder.array(Schema.STRING_SCHEMA).build()) - .field("map", SchemaBuilder.map(Schema.STRING_SCHEMA, Schema.STRING_SCHEMA).build()) - .field("struct", structSchema) - .field("decimal", Decimal.schema(2)) - .field("date", Date.SCHEMA) - .field("time", Time.SCHEMA) - .field("timestamp", Timestamp.SCHEMA) + @Test + public void testBuildMappingSetsNoDefaultValueForStrings() throws IOException { + Schema schema = SchemaBuilder + .struct() + .name("record") + .field("string", SchemaBuilder.string().defaultValue("0").build()) + .build(); + + JsonObject result = runTest(schema); + + assertNull(result.getAsJsonObject("properties").getAsJsonObject("string").get("null_value")); + } + + private Schema createSchema() { + return createSchemaBuilder("record") + .field("struct", createSchemaBuilder("inner").build()) .build(); } - private Schema createInnerSchema() { - return SchemaBuilder.struct().name("inner") + private SchemaBuilder createSchemaBuilder(String name) { + return SchemaBuilder.struct().name(name) .field("boolean", Schema.BOOLEAN_SCHEMA) .field("bytes", Schema.BYTES_SCHEMA) .field("int8", Schema.INT8_SCHEMA) @@ -157,16 +140,21 @@ private Schema createInnerSchema() { .field("float64", Schema.FLOAT64_SCHEMA) .field("string", Schema.STRING_SCHEMA) .field("array", SchemaBuilder.array(Schema.STRING_SCHEMA).build()) - .field("map", SchemaBuilder.map(Schema.STRING_SCHEMA, Schema.STRING_SCHEMA).build()) + .field("map", SchemaBuilder.map(Schema.STRING_SCHEMA, Schema.INT32_SCHEMA).build()) .field("decimal", Decimal.schema(2)) .field("date", Date.SCHEMA) .field("time", Time.SCHEMA) - .field("timestamp", Timestamp.SCHEMA) - .build(); + .field("timestamp", Timestamp.SCHEMA); + } + + private static JsonObject runTest(Schema schema) throws IOException { + XContentBuilder builder = Mapping.buildMapping(schema); + builder.flush(); + ByteArrayOutputStream stream = (ByteArrayOutputStream) builder.getOutputStream(); + return (JsonObject) JsonParser.parseString(stream.toString()); } - @SuppressWarnings("unchecked") - private void verifyMapping(Schema schema, JsonObject mapping) throws Exception { + private void verifyMapping(Schema schema, JsonObject mapping) { String schemaName = schema.name(); Object type = mapping.get("type"); if (schemaName != null) { @@ -174,15 +162,15 @@ private void verifyMapping(Schema schema, JsonObject mapping) throws Exception { case Date.LOGICAL_NAME: case Time.LOGICAL_NAME: case Timestamp.LOGICAL_NAME: - assertEquals("\"" + ElasticsearchSinkConnectorConstants.DATE_TYPE + "\"", type.toString()); + assertEquals("\"" + Mapping.DATE_TYPE + "\"", type.toString()); return; case Decimal.LOGICAL_NAME: - assertEquals("\"" + ElasticsearchSinkConnectorConstants.DOUBLE_TYPE + "\"", type.toString()); + assertEquals("\"" + Mapping.DOUBLE_TYPE + "\"", type.toString()); return; } } - DataConverter converter = new DataConverter(true, BehaviorOnNullValues.IGNORE); + DataConverter converter = new DataConverter(new ElasticsearchSinkConnectorConfig(ElasticsearchSinkConnectorConfigTest.addNecessaryProps(new HashMap<>()))); Schema.Type schemaType = schema.type(); switch (schemaType) { case ARRAY: @@ -191,8 +179,8 @@ private void verifyMapping(Schema schema, JsonObject mapping) throws Exception { case MAP: Schema newSchema = converter.preProcessSchema(schema); JsonObject mapProperties = mapping.get("properties").getAsJsonObject(); - verifyMapping(newSchema.keySchema(), mapProperties.get(ElasticsearchSinkConnectorConstants.MAP_KEY).getAsJsonObject()); - verifyMapping(newSchema.valueSchema(), mapProperties.get(ElasticsearchSinkConnectorConstants.MAP_VALUE).getAsJsonObject()); + verifyMapping(newSchema.keySchema(), mapProperties.get(KEY_FIELD).getAsJsonObject()); + verifyMapping(newSchema.valueSchema(), mapProperties.get(VALUE_FIELD).getAsJsonObject()); break; case STRUCT: JsonObject properties = mapping.get("properties").getAsJsonObject(); @@ -201,7 +189,7 @@ private void verifyMapping(Schema schema, JsonObject mapping) throws Exception { } break; default: - assertEquals("\"" + Mapping.getElasticsearchType(client, schemaType) + "\"", type.toString()); + assertEquals("\"" + Mapping.getElasticsearchType(schemaType) + "\"", type.toString()); } } } diff --git a/src/test/java/io/confluent/connect/elasticsearch/RetryUtilTest.java b/src/test/java/io/confluent/connect/elasticsearch/RetryUtilTest.java index b91549365..45438840b 100644 --- a/src/test/java/io/confluent/connect/elasticsearch/RetryUtilTest.java +++ b/src/test/java/io/confluent/connect/elasticsearch/RetryUtilTest.java @@ -14,12 +14,34 @@ */ package io.confluent.connect.elasticsearch; +import java.io.IOException; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.connect.errors.ConnectException; +import org.junit.Before; import org.junit.Test; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; public class RetryUtilTest { + + private int timesThrown; + + @Before + public void setup() { + timesThrown = 0; + } + + @Test + public void computeRetryBackoffForNegativeAttempts() { + assertComputeRetryInRange(0, 10L); + assertEquals(10L, RetryUtil.computeRandomRetryWaitTimeInMillis(-1, 10L)); + } + @Test public void computeRetryBackoffForValidRanges() { assertComputeRetryInRange(10, 10L); @@ -45,6 +67,40 @@ public void computeNonRandomRetryTimes() { assertEquals(3200L, RetryUtil.computeRetryWaitTimeInMillis(5, 100L)); } + @Test + public void testCallWithRetriesNoRetries() throws Exception { + MockTime mockClock = new MockTime(); + long expectedTime = mockClock.milliseconds(); + + assertTrue(RetryUtil.callWithRetries("test", () -> testFunction(0), 3, 100, mockClock)); + assertEquals(expectedTime, mockClock.milliseconds()); + } + + @Test + public void testCallWithRetriesSomeRetries() throws Exception { + MockTime mockClock = spy(new MockTime()); + + assertTrue(RetryUtil.callWithRetries("test", () -> testFunction(2), 3, 100, mockClock)); + verify(mockClock, times(2)).sleep(anyLong()); + } + + @Test(expected = ConnectException.class) + public void testCallWithRetriesExhaustedRetries() throws Exception { + MockTime mockClock = new MockTime(); + + assertTrue(RetryUtil.callWithRetries("test", () -> testFunction(4), 3, 100, mockClock)); + verify(mockClock, times(3)).sleep(anyLong()); + } + + private boolean testFunction(int timesToThrow) throws IOException { + if (timesThrown < timesToThrow) { + timesThrown++; + throw new IOException("oh no i iz borke, plz retry"); + } + + return true; + } + protected void assertComputeRetryInRange(int retryAttempts, long retryBackoffMs) { for (int i = 0; i != 20; ++i) { for (int retries = 0; retries <= retryAttempts; ++retries) { diff --git a/src/test/java/io/confluent/connect/elasticsearch/TestUtils.java b/src/test/java/io/confluent/connect/elasticsearch/TestUtils.java deleted file mode 100644 index bf06ef96c..000000000 --- a/src/test/java/io/confluent/connect/elasticsearch/TestUtils.java +++ /dev/null @@ -1,103 +0,0 @@ -package io.confluent.connect.elasticsearch; - -import static org.junit.Assert.assertEquals; - -import com.google.gson.JsonObject; -import io.confluent.connect.elasticsearch.DataConverter.BehaviorOnNullValues; -import org.apache.kafka.connect.data.Date; -import org.apache.kafka.connect.data.Decimal; -import org.apache.kafka.connect.data.Field; -import org.apache.kafka.connect.data.Schema; -import org.apache.kafka.connect.data.SchemaBuilder; -import org.apache.kafka.connect.data.Time; -import org.apache.kafka.connect.data.Timestamp; - -public class TestUtils { - - - public static Schema createSchema() { - Schema structSchema = createInnerSchema(); - return SchemaBuilder.struct().name("record") - .field("boolean", Schema.BOOLEAN_SCHEMA) - .field("bytes", Schema.BYTES_SCHEMA) - .field("int8", Schema.INT8_SCHEMA) - .field("int16", Schema.INT16_SCHEMA) - .field("int32", Schema.INT32_SCHEMA) - .field("int64", Schema.INT64_SCHEMA) - .field("float32", Schema.FLOAT32_SCHEMA) - .field("float64", Schema.FLOAT64_SCHEMA) - .field("string", Schema.STRING_SCHEMA) - .field("array", SchemaBuilder.array(Schema.STRING_SCHEMA).build()) - .field("map", SchemaBuilder.map(Schema.STRING_SCHEMA, Schema.STRING_SCHEMA).build()) - .field("struct", structSchema) - .field("decimal", Decimal.schema(2)) - .field("date", Date.SCHEMA) - .field("time", Time.SCHEMA) - .field("timestamp", Timestamp.SCHEMA) - .build(); - } - - private static Schema createInnerSchema() { - return SchemaBuilder.struct().name("inner") - .field("boolean", Schema.BOOLEAN_SCHEMA) - .field("bytes", Schema.BYTES_SCHEMA) - .field("int8", Schema.INT8_SCHEMA) - .field("int16", Schema.INT16_SCHEMA) - .field("int32", Schema.INT32_SCHEMA) - .field("int64", Schema.INT64_SCHEMA) - .field("float32", Schema.FLOAT32_SCHEMA) - .field("float64", Schema.FLOAT64_SCHEMA) - .field("string", Schema.STRING_SCHEMA) - .field("array", SchemaBuilder.array(Schema.STRING_SCHEMA).build()) - .field("map", SchemaBuilder.map(Schema.STRING_SCHEMA, Schema.STRING_SCHEMA).build()) - .field("decimal", Decimal.schema(2)) - .field("date", Date.SCHEMA) - .field("time", Time.SCHEMA) - .field("timestamp", Timestamp.SCHEMA) - .build(); - } - @SuppressWarnings("unchecked") - public static void verifyMapping(ElasticsearchClient client, Schema schema, JsonObject mapping) throws Exception { - String schemaName = schema.name(); - Object type = mapping.get("type"); - if (schemaName != null) { - switch (schemaName) { - case Date.LOGICAL_NAME: - case Time.LOGICAL_NAME: - case Timestamp.LOGICAL_NAME: - assertEquals("\"" + ElasticsearchSinkConnectorConstants.DATE_TYPE + "\"", - type.toString()); - return; - case Decimal.LOGICAL_NAME: - assertEquals("\"" + ElasticsearchSinkConnectorConstants.DOUBLE_TYPE + "\"", - type.toString()); - return; - } - } - - DataConverter converter = new DataConverter(true, BehaviorOnNullValues.IGNORE); - Schema.Type schemaType = schema.type(); - switch (schemaType) { - case ARRAY: - verifyMapping(client, schema.valueSchema(), mapping); - break; - case MAP: - Schema newSchema = converter.preProcessSchema(schema); - JsonObject mapProperties = mapping.get("properties").getAsJsonObject(); - verifyMapping(client, newSchema.keySchema(), - mapProperties.get(ElasticsearchSinkConnectorConstants.MAP_KEY).getAsJsonObject()); - verifyMapping(client, newSchema.valueSchema(), - mapProperties.get(ElasticsearchSinkConnectorConstants.MAP_VALUE).getAsJsonObject()); - break; - case STRUCT: - JsonObject properties = mapping.get("properties").getAsJsonObject(); - for (Field field : schema.fields()) { - verifyMapping(client, field.schema(), properties.get(field.name()).getAsJsonObject()); - } - break; - default: - assertEquals("\"" + Mapping.getElasticsearchType(client, schemaType) + "\"", - type.toString()); - } - } -} diff --git a/src/test/java/io/confluent/connect/elasticsearch/ValidatorTest.java b/src/test/java/io/confluent/connect/elasticsearch/ValidatorTest.java index 49984fa5a..fed305514 100644 --- a/src/test/java/io/confluent/connect/elasticsearch/ValidatorTest.java +++ b/src/test/java/io/confluent/connect/elasticsearch/ValidatorTest.java @@ -19,49 +19,70 @@ import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.BATCH_SIZE_CONFIG; import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.CONNECTION_PASSWORD_CONFIG; import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.CONNECTION_URL_CONFIG; - -import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.SECURITY_PROTOCOL_CONFIG; -import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.SSL_CONFIG_PREFIX; import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.CONNECTION_USERNAME_CONFIG; import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.FLUSH_TIMEOUT_MS_CONFIG; import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.IGNORE_KEY_CONFIG; import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.IGNORE_KEY_TOPICS_CONFIG; import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.IGNORE_SCHEMA_CONFIG; import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.IGNORE_SCHEMA_TOPICS_CONFIG; +import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.KERBEROS_KEYTAB_PATH_CONFIG; import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.LINGER_MS_CONFIG; import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.MAX_BUFFERED_RECORDS_CONFIG; import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.MAX_IN_FLIGHT_REQUESTS_CONFIG; import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.PROXY_HOST_CONFIG; import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.PROXY_PASSWORD_CONFIG; import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.PROXY_USERNAME_CONFIG; -import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.TYPE_NAME_CONFIG; +import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.SECURITY_PROTOCOL_CONFIG; +import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.SSL_CONFIG_PREFIX; +import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.KERBEROS_PRINCIPAL_CONFIG; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; - +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.SecurityProtocol; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; import java.util.HashMap; import java.util.Map; import org.apache.kafka.common.config.Config; import org.apache.kafka.common.config.ConfigValue; import org.apache.kafka.common.config.SslConfigs; +import org.elasticsearch.ElasticsearchStatusException; +import org.elasticsearch.client.RequestOptions; +import org.elasticsearch.client.RestHighLevelClient; +import org.elasticsearch.rest.RestStatus; import org.junit.Before; import org.junit.Test; public class ValidatorTest { private Map props; + private RestHighLevelClient mockClient; private Validator validator; @Before - public void setup() { - props = new HashMap<>(); - props.put(TYPE_NAME_CONFIG, "type"); - props.put(CONNECTION_URL_CONFIG, "localhost:8080"); + public void setup() throws IOException { + props = ElasticsearchSinkConnectorConfigTest.addNecessaryProps(new HashMap<>()); + + mockClient = mock(RestHighLevelClient.class); + when(mockClient.ping(any(RequestOptions.class))).thenReturn(true); + } + + @Test + public void testInvalidIndividualConfigs() { + validator = new Validator(new HashMap<>(), () -> mockClient); + Config result = validator.validate(); + assertHasErrorMessage(result, CONNECTION_URL_CONFIG, "Missing required configuration"); } @Test public void testInvalidCredentials() { props.put(CONNECTION_USERNAME_CONFIG, "username"); - validator = new Validator(props); + validator = new Validator(props, () -> mockClient); Config result = validator.validate(); assertHasErrorMessage(result, CONNECTION_USERNAME_CONFIG, "must be set"); @@ -75,10 +96,18 @@ public void testInvalidCredentials() { assertHasErrorMessage(result, CONNECTION_PASSWORD_CONFIG, "must be set"); } + @Test + public void testClientThrowsElasticsearchStatusException() throws IOException { + when(mockClient.ping(any(RequestOptions.class))).thenThrow(new ElasticsearchStatusException("Deleted resource.", RestStatus.GONE)); + validator = new Validator(props, () -> mockClient); + Config result = validator.validate(); + assertHasErrorMessage(result, CONNECTION_URL_CONFIG, "Could not connect to Elasticsearch. Error message: Deleted resource."); + } + @Test public void testValidCredentials() { // username and password not set - validator = new Validator(props); + validator = new Validator(props, () -> mockClient); Config result = validator.validate(); assertNoErrors(result); @@ -86,7 +115,7 @@ public void testValidCredentials() { // both set props.put(CONNECTION_USERNAME_CONFIG, "username"); props.put(CONNECTION_PASSWORD_CONFIG, "password"); - validator = new Validator(props); + validator = new Validator(props, () -> mockClient); result = validator.validate(); assertNoErrors(result); @@ -98,7 +127,7 @@ public void testInvalidIgnoreConfigs() { props.put(IGNORE_KEY_TOPICS_CONFIG, "some,topics"); props.put(IGNORE_SCHEMA_CONFIG, "true"); props.put(IGNORE_SCHEMA_TOPICS_CONFIG, "some,other,topics"); - validator = new Validator(props); + validator = new Validator(props, () -> mockClient); Config result = validator.validate(); assertHasErrorMessage(result, IGNORE_KEY_CONFIG, "is true"); @@ -112,7 +141,7 @@ public void testValidIgnoreConfigs() { // topics configs not set props.put(IGNORE_KEY_CONFIG, "true"); props.put(IGNORE_SCHEMA_CONFIG, "true"); - validator = new Validator(props); + validator = new Validator(props, () -> mockClient); Config result = validator.validate(); assertNoErrors(result); @@ -122,17 +151,72 @@ public void testValidIgnoreConfigs() { props.put(IGNORE_KEY_TOPICS_CONFIG, "some,topics"); props.put(IGNORE_SCHEMA_CONFIG, "false"); props.put(IGNORE_SCHEMA_TOPICS_CONFIG, "some,other,topics"); - validator = new Validator(props); + validator = new Validator(props, () -> mockClient); result = validator.validate(); assertNoErrors(result); } + @Test + public void testInvalidKerberos() throws IOException { + props.put(KERBEROS_PRINCIPAL_CONFIG, "principal"); + validator = new Validator(props, () -> mockClient); + + Config result = validator.validate(); + assertHasErrorMessage(result, KERBEROS_PRINCIPAL_CONFIG, "must be set"); + assertHasErrorMessage(result, KERBEROS_KEYTAB_PATH_CONFIG, "must be set"); + + // proxy + Path keytab = Files.createTempFile("es", ".keytab"); + props.put(KERBEROS_PRINCIPAL_CONFIG, "principal"); + props.put(KERBEROS_KEYTAB_PATH_CONFIG, keytab.toString()); + props.put(PROXY_HOST_CONFIG, "proxy.com"); + validator = new Validator(props, () -> mockClient); + + result = validator.validate(); + assertHasErrorMessage(result, KERBEROS_PRINCIPAL_CONFIG, "not supported with proxy settings"); + assertHasErrorMessage(result, KERBEROS_KEYTAB_PATH_CONFIG, "not supported with proxy settings"); + assertHasErrorMessage(result, PROXY_HOST_CONFIG, "not supported with proxy settings"); + + // basic credentials + props.remove(PROXY_HOST_CONFIG); + props.put(CONNECTION_USERNAME_CONFIG, "username"); + props.put(CONNECTION_PASSWORD_CONFIG, "password"); + validator = new Validator(props, () -> mockClient); + + result = validator.validate(); + assertHasErrorMessage(result, KERBEROS_PRINCIPAL_CONFIG, "Either only Kerberos"); + assertHasErrorMessage(result, KERBEROS_KEYTAB_PATH_CONFIG, "Either only Kerberos"); + assertHasErrorMessage(result, CONNECTION_USERNAME_CONFIG, "Either only Kerberos"); + assertHasErrorMessage(result, CONNECTION_PASSWORD_CONFIG, "Either only Kerberos"); + + keytab.toFile().delete(); + } + + @Test + public void testValidKerberos() throws IOException { + // kerberos configs not set + validator = new Validator(props, () -> mockClient); + + Config result = validator.validate(); + assertNoErrors(result); + + // kerberos configs both false + Path keytab = Files.createTempFile("es", ".keytab"); + props.put(KERBEROS_PRINCIPAL_CONFIG, "principal"); + props.put(KERBEROS_KEYTAB_PATH_CONFIG, keytab.toString()); + validator = new Validator(props, () -> mockClient); + + result = validator.validate(); + assertNoErrors(result); + keytab.toFile().delete(); + } + @Test public void testInvalidLingerMs() { props.put(LINGER_MS_CONFIG, "1001"); props.put(FLUSH_TIMEOUT_MS_CONFIG, "1000"); - validator = new Validator(props); + validator = new Validator(props, () -> mockClient); Config result = validator.validate(); assertHasErrorMessage(result, LINGER_MS_CONFIG, "can not be larger than"); @@ -143,7 +227,7 @@ public void testInvalidLingerMs() { public void testValidLingerMs() { props.put(LINGER_MS_CONFIG, "999"); props.put(FLUSH_TIMEOUT_MS_CONFIG, "1000"); - validator = new Validator(props); + validator = new Validator(props, () -> mockClient); Config result = validator.validate(); assertNoErrors(result); @@ -154,7 +238,7 @@ public void testInvalidMaxBufferedRecords() { props.put(MAX_BUFFERED_RECORDS_CONFIG, "1"); props.put(BATCH_SIZE_CONFIG, "2"); props.put(MAX_IN_FLIGHT_REQUESTS_CONFIG, "2"); - validator = new Validator(props); + validator = new Validator(props, () -> mockClient); Config result = validator.validate(); assertHasErrorMessage(result, MAX_BUFFERED_RECORDS_CONFIG, "must be larger than or equal to"); @@ -167,7 +251,7 @@ public void testValidMaxBufferedRecords() { props.put(MAX_BUFFERED_RECORDS_CONFIG, "5"); props.put(BATCH_SIZE_CONFIG, "2"); props.put(MAX_IN_FLIGHT_REQUESTS_CONFIG, "2"); - validator = new Validator(props); + validator = new Validator(props, () -> mockClient); Config result = validator.validate(); assertNoErrors(result); @@ -178,7 +262,7 @@ public void testInvalidProxy() { props.put(PROXY_HOST_CONFIG, ""); props.put(PROXY_USERNAME_CONFIG, "username"); props.put(PROXY_PASSWORD_CONFIG, "password"); - validator = new Validator(props); + validator = new Validator(props, () -> mockClient); Config result = validator.validate(); assertHasErrorMessage(result, PROXY_HOST_CONFIG, " must be set to use"); @@ -189,7 +273,7 @@ public void testInvalidProxy() { props.put(PROXY_HOST_CONFIG, "proxy"); props.put(PROXY_PASSWORD_CONFIG, "password"); - validator = new Validator(props); + validator = new Validator(props, () -> mockClient); result = validator.validate(); assertHasErrorMessage(result, PROXY_USERNAME_CONFIG, "Either both or neither"); @@ -199,7 +283,7 @@ public void testInvalidProxy() { @Test public void testValidProxy() { props.put(PROXY_HOST_CONFIG, "proxy"); - validator = new Validator(props); + validator = new Validator(props, () -> mockClient); Config result = validator.validate(); assertNoErrors(result); @@ -207,7 +291,7 @@ public void testValidProxy() { props.put(PROXY_HOST_CONFIG, "proxy"); props.put(PROXY_USERNAME_CONFIG, "password"); props.put(PROXY_PASSWORD_CONFIG, "password"); - validator = new Validator(props); + validator = new Validator(props, () -> mockClient); result = validator.validate(); assertNoErrors(result); @@ -217,7 +301,7 @@ public void testValidProxy() { public void testInvalidSsl() { // no SSL props.put(SECURITY_PROTOCOL_CONFIG, SecurityProtocol.SSL.name()); - validator = new Validator(props); + validator = new Validator(props, () -> mockClient); Config result = validator.validate(); assertHasErrorMessage(result, SECURITY_PROTOCOL_CONFIG, "At least these SSL configs "); @@ -228,7 +312,7 @@ public void testInvalidSsl() { props.put(SSL_CONFIG_PREFIX + SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, "b"); props.put(SSL_CONFIG_PREFIX + SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, "c"); props.put(SSL_CONFIG_PREFIX + SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, "d"); - validator = new Validator(props); + validator = new Validator(props, () -> mockClient); result = validator.validate(); assertHasErrorMessage(result, SECURITY_PROTOCOL_CONFIG, "to use SSL configs"); @@ -238,7 +322,7 @@ public void testInvalidSsl() { public void testValidSsl() { // no SSL props.put(SECURITY_PROTOCOL_CONFIG, SecurityProtocol.PLAINTEXT.name()); - validator = new Validator(props); + validator = new Validator(props, () -> mockClient); Config result = validator.validate(); assertNoErrors(result); @@ -249,12 +333,38 @@ public void testValidSsl() { props.put(SSL_CONFIG_PREFIX + SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, "b"); props.put(SSL_CONFIG_PREFIX + SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, "c"); props.put(SSL_CONFIG_PREFIX + SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, "d"); - validator = new Validator(props); + validator = new Validator(props, () -> mockClient); result = validator.validate(); assertNoErrors(result); } + @Test + public void testValidConnection() { + validator = new Validator(props, () -> mockClient); + + Config result = validator.validate(); + assertNoErrors(result); + } + + @Test + public void testInvalidConnection() throws IOException { + when(mockClient.ping(eq(RequestOptions.DEFAULT))).thenReturn(false); + validator = new Validator(props, () -> mockClient); + + Config result = validator.validate(); + assertHasErrorMessage(result, CONNECTION_URL_CONFIG, "Could not connect to Elasticsearch."); + } + + @Test + public void testInvalidConnectionThrows() throws IOException { + when(mockClient.ping(eq(RequestOptions.DEFAULT))).thenThrow(new IOException("i iz fake")); + validator = new Validator(props, () -> mockClient); + + Config result = validator.validate(); + assertHasErrorMessage(result, CONNECTION_URL_CONFIG, "Could not connect to Elasticsearch."); + } + private static void assertHasErrorMessage(Config config, String property, String msg) { for (ConfigValue configValue : config.configValues()) { if (configValue.name().equals(property)) { diff --git a/src/test/java/io/confluent/connect/elasticsearch/bulk/BulkProcessorTest.java b/src/test/java/io/confluent/connect/elasticsearch/bulk/BulkProcessorTest.java deleted file mode 100644 index ba36c62c2..000000000 --- a/src/test/java/io/confluent/connect/elasticsearch/bulk/BulkProcessorTest.java +++ /dev/null @@ -1,685 +0,0 @@ -/* - * 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.connect.elasticsearch.bulk; - -import com.google.gson.Gson; -import com.google.gson.JsonArray; -import com.google.gson.JsonObject; -import io.confluent.connect.elasticsearch.IndexableRecord; -import io.confluent.connect.elasticsearch.Key; -import io.searchbox.client.JestResult; -import io.searchbox.core.BulkResult; -import io.searchbox.core.BulkResult.BulkResultItem; -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; -import org.apache.kafka.common.utils.Time; -import org.apache.kafka.connect.data.Schema; -import org.apache.kafka.connect.errors.ConnectException; -import org.apache.kafka.connect.sink.ErrantRecordReporter; -import org.apache.kafka.connect.sink.SinkRecord; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.LinkedList; -import java.util.List; -import java.util.Queue; -import java.util.concurrent.ExecutionException; - -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.containsString; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertThrows; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - -import static io.confluent.connect.elasticsearch.bulk.BulkProcessor.BehaviorOnMalformedDoc; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyLong; -import static org.mockito.ArgumentMatchers.eq; -import static org.mockito.Mockito.doAnswer; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.never; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; - -public class BulkProcessorTest { - - private static final String INDEX = "topic"; - private static final String TYPE = "type"; - - private static class Expectation { - final List request; - final BulkResponse response; - - private Expectation(List request, BulkResponse response) { - this.request = request; - this.response = response; - } - } - - private static final class Client implements BulkClient> { - private final Queue expectQ = new LinkedList<>(); - private volatile boolean executeMetExpectations = true; - - @Override - public List bulkRequest(List batch) { - List ids = new ArrayList<>(batch.size()); - for (IndexableRecord id : batch) { - ids.add(Integer.valueOf(id.key.id)); - } - return ids; - } - - public void expect(List ids, BulkResponse response) { - expectQ.add(new Expectation(ids, response)); - } - - public boolean expectationsMet() { - return expectQ.isEmpty() && executeMetExpectations; - } - - @Override - public BulkResponse execute(List request) { - final Expectation expectation; - try { - expectation = expectQ.remove(); - assertEquals(expectation.request, request); - } catch (Throwable t) { - executeMetExpectations = false; - throw t; - } - executeMetExpectations &= true; - return expectation.response; - } - } - - Client client; - - @Before - public void createClient() { - client = new Client(); - } - - @After - public void checkClient() { - assertTrue(client.expectationsMet()); - } - - @Test - public void batchingAndLingering() throws InterruptedException, ExecutionException { - final int maxBufferedRecords = 100; - final int maxInFlightBatches = 5; - final int batchSize = 5; - final int lingerMs = 5; - final int maxRetries = 0; - final int retryBackoffMs = 0; - final BehaviorOnMalformedDoc behaviorOnMalformedDoc = BehaviorOnMalformedDoc.DEFAULT; - final ErrantRecordReporter reporter = mock(ErrantRecordReporter.class); - - final BulkProcessor bulkProcessor = new BulkProcessor<>( - Time.SYSTEM, - client, - maxBufferedRecords, - maxInFlightBatches, - batchSize, - lingerMs, - maxRetries, - retryBackoffMs, - behaviorOnMalformedDoc, - reporter - ); - - final int addTimeoutMs = 10; - for (int i = 1; i < 13; i++) { - bulkProcessor.add(indexableRecord(i), sinkRecord(), addTimeoutMs); - } - - client.expect(Arrays.asList(1, 2, 3, 4, 5), BulkResponse.success()); - client.expect(Arrays.asList(6, 7, 8, 9, 10), BulkResponse.success()); - client.expect(Arrays.asList(11, 12), BulkResponse.success()); // batch not full, but upon linger timeout - assertTrue(bulkProcessor.submitBatchWhenReady().get().succeeded); - assertTrue(bulkProcessor.submitBatchWhenReady().get().succeeded); - assertTrue(bulkProcessor.submitBatchWhenReady().get().succeeded); - assertTrue(bulkProcessor.recordsToReportOnError.isEmpty()); - verify(reporter, never()).report(eq(sinkRecord()), any()); - } - - @Test - public void flushing() { - final int maxBufferedRecords = 100; - final int maxInFlightBatches = 5; - final int batchSize = 5; - final int lingerMs = 100000; // super high on purpose to make sure flush is what's causing the request - final int maxRetries = 0; - final int retryBackoffMs = 0; - final BehaviorOnMalformedDoc behaviorOnMalformedDoc = BehaviorOnMalformedDoc.DEFAULT; - final ErrantRecordReporter reporter = mock(ErrantRecordReporter.class); - - final BulkProcessor bulkProcessor = new BulkProcessor<>( - Time.SYSTEM, - client, - maxBufferedRecords, - maxInFlightBatches, - batchSize, - lingerMs, - maxRetries, - retryBackoffMs, - behaviorOnMalformedDoc, - reporter - ); - - client.expect(Arrays.asList(1, 2, 3), BulkResponse.success()); - - bulkProcessor.start(); - - final int addTimeoutMs = 10; - bulkProcessor.add(indexableRecord(1), sinkRecord(), addTimeoutMs); - bulkProcessor.add(indexableRecord(2), sinkRecord(), addTimeoutMs); - bulkProcessor.add(indexableRecord(3), sinkRecord(), addTimeoutMs); - - assertFalse(client.expectationsMet()); - - final int flushTimeoutMs = 100; - bulkProcessor.flush(flushTimeoutMs); - assertTrue(bulkProcessor.recordsToReportOnError.isEmpty()); - verify(reporter, never()).report(eq(sinkRecord()), any()); - } - - @Test - public void addBlocksWhenBufferFull() { - final int maxBufferedRecords = 1; - final int maxInFlightBatches = 1; - final int batchSize = 1; - final int lingerMs = 10; - final int maxRetries = 0; - final int retryBackoffMs = 0; - final BehaviorOnMalformedDoc behaviorOnMalformedDoc = BehaviorOnMalformedDoc.DEFAULT; - final ErrantRecordReporter reporter = mock(ErrantRecordReporter.class); - - final BulkProcessor bulkProcessor = new BulkProcessor<>( - Time.SYSTEM, - client, - maxBufferedRecords, - maxInFlightBatches, - batchSize, - lingerMs, - maxRetries, - retryBackoffMs, - behaviorOnMalformedDoc, - reporter - ); - - final int addTimeoutMs = 10; - bulkProcessor.add(indexableRecord(42), sinkRecord(), addTimeoutMs); - assertEquals(1, bulkProcessor.bufferedRecords()); - try { - // BulkProcessor not started, so this add should timeout & throw - bulkProcessor.add(indexableRecord(43), sinkRecord(), addTimeoutMs); - fail(); - } catch (ConnectException good) { - } - assertEquals(1, bulkProcessor.recordsToReportOnError.size()); - verify(reporter, never()).report(eq(sinkRecord()), any()); - } - - @Test - public void retriableErrors() throws InterruptedException, ExecutionException { - final int maxBufferedRecords = 100; - final int maxInFlightBatches = 5; - final int batchSize = 2; - final int lingerMs = 5; - final int maxRetries = 3; - final int retryBackoffMs = 1; - final BehaviorOnMalformedDoc behaviorOnMalformedDoc = BehaviorOnMalformedDoc.DEFAULT; - final ErrantRecordReporter reporter = mock(ErrantRecordReporter.class); - - client.expect(Arrays.asList(42, 43), BulkResponse.failure(true, "a retiable error", getFailedRecords(42, 43))); - client.expect(Arrays.asList(42, 43), BulkResponse.failure(true, "a retriable error again", getFailedRecords(42, 43))); - client.expect(Arrays.asList(42, 43), BulkResponse.success()); - - final BulkProcessor bulkProcessor = new BulkProcessor<>( - Time.SYSTEM, - client, - maxBufferedRecords, - maxInFlightBatches, - batchSize, - lingerMs, - maxRetries, - retryBackoffMs, - behaviorOnMalformedDoc, - reporter - ); - - final int addTimeoutMs = 10; - bulkProcessor.add(indexableRecord(42), sinkRecord(), addTimeoutMs); - bulkProcessor.add(indexableRecord(43), sinkRecord(), addTimeoutMs); - - assertTrue(bulkProcessor.submitBatchWhenReady().get().succeeded); - assertTrue(bulkProcessor.recordsToReportOnError.isEmpty()); - verify(reporter, never()).report(eq(sinkRecord()), any()); - } - - @Test - public void retriableErrorsHitMaxRetries() throws InterruptedException { - final int maxBufferedRecords = 100; - final int maxInFlightBatches = 5; - final int batchSize = 2; - final int lingerMs = 5; - final int maxRetries = 2; - final int retryBackoffMs = 1; - final String errorInfo = "a final retriable error again"; - final BehaviorOnMalformedDoc behaviorOnMalformedDoc = BehaviorOnMalformedDoc.DEFAULT; - final ErrantRecordReporter reporter = mock(ErrantRecordReporter.class); - - client.expect(Arrays.asList(42, 43), BulkResponse.failure(true, "a retriable error", getFailedRecords(42, 43))); - client.expect(Arrays.asList(42, 43), BulkResponse.failure(true, "a retriable error again", getFailedRecords(42, 43))); - client.expect(Arrays.asList(42, 43), BulkResponse.failure(true, errorInfo, getFailedRecords(42, 43))); - - final BulkProcessor bulkProcessor = new BulkProcessor<>( - Time.SYSTEM, - client, - maxBufferedRecords, - maxInFlightBatches, - batchSize, - lingerMs, - maxRetries, - retryBackoffMs, - behaviorOnMalformedDoc, - reporter - ); - - final int addTimeoutMs = 10; - bulkProcessor.add(indexableRecord(42), sinkRecord(), addTimeoutMs); - bulkProcessor.add(indexableRecord(43), sinkRecord(), addTimeoutMs); - - try { - bulkProcessor.submitBatchWhenReady().get(); - fail(); - } catch (ExecutionException e) { - assertTrue(e.getCause().getMessage().contains(errorInfo)); - assertTrue(bulkProcessor.recordsToReportOnError.isEmpty()); - verify(reporter, never()).report(eq(sinkRecord()), any()); - } - } - - @Test - public void unretriableErrors() throws InterruptedException { - final int maxBufferedRecords = 100; - final int maxInFlightBatches = 5; - final int batchSize = 2; - final int lingerMs = 5; - final int maxRetries = 3; - final int retryBackoffMs = 1; - final BehaviorOnMalformedDoc behaviorOnMalformedDoc = BehaviorOnMalformedDoc.DEFAULT; - final ErrantRecordReporter reporter = mock(ErrantRecordReporter.class); - - final String errorInfo = "an unretriable error"; - client.expect(Arrays.asList(42, 43), BulkResponse.failure(false, errorInfo, getFailedRecords(42, 43))); - - final BulkProcessor bulkProcessor = new BulkProcessor<>( - Time.SYSTEM, - client, - maxBufferedRecords, - maxInFlightBatches, - batchSize, - lingerMs, - maxRetries, - retryBackoffMs, - behaviorOnMalformedDoc, - reporter - ); - - final int addTimeoutMs = 10; - bulkProcessor.add(indexableRecord(42), sinkRecord(), addTimeoutMs); - bulkProcessor.add(indexableRecord(43), sinkRecord(), addTimeoutMs); - - try { - bulkProcessor.submitBatchWhenReady().get(); - fail(); - } catch (ExecutionException e) { - assertTrue(e.getCause().getMessage().contains(errorInfo)); - assertTrue(bulkProcessor.recordsToReportOnError.isEmpty()); - verify(reporter, never()).report(eq(sinkRecord()), any()); - } - } - - @Test - public void failOnMalformedDoc() { - final int maxBufferedRecords = 100; - final int maxInFlightBatches = 5; - final int batchSize = 2; - final int lingerMs = 5; - final int maxRetries = 3; - final int retryBackoffMs = 1; - final BehaviorOnMalformedDoc behaviorOnMalformedDoc = BehaviorOnMalformedDoc.FAIL; - final ErrantRecordReporter reporter = mock(ErrantRecordReporter.class); - - final String errorInfo = " [{\"type\":\"mapper_parsing_exception\",\"reason\":\"failed to parse\"," + - "\"caused_by\":{\"type\":\"illegal_argument_exception\",\"reason\":\"object\n" + - " field starting or ending with a [.] makes object resolution ambiguous: [avjpz{{.}}wjzse{{..}}gal9d]\"}}]"; - client.expect(Arrays.asList(42, 43), BulkResponse.failure(false, errorInfo, getFailedRecords(42, 43))); - - final BulkProcessor bulkProcessor = new BulkProcessor<>( - Time.SYSTEM, - client, - maxBufferedRecords, - maxInFlightBatches, - batchSize, - lingerMs, - maxRetries, - retryBackoffMs, - behaviorOnMalformedDoc, - reporter - ); - - bulkProcessor.start(); - - bulkProcessor.add(indexableRecord(42), sinkRecord(),1); - bulkProcessor.add(indexableRecord(43), sinkRecord(), 1); - - try { - final int flushTimeoutMs = 1000; - bulkProcessor.flush(flushTimeoutMs); - fail(); - } catch(ConnectException e) { - // expected - assertTrue(e.getMessage().contains("mapper_parsing_exception")); - assertTrue(bulkProcessor.recordsToReportOnError.isEmpty()); - verify(reporter, never()).report(eq(sinkRecord()), any()); - } - } - - @Test - public void ignoreOrWarnOnMalformedDoc() { - final int maxBufferedRecords = 100; - final int maxInFlightBatches = 5; - final int batchSize = 2; - final int lingerMs = 5; - final int maxRetries = 3; - final int retryBackoffMs = 1; - final ErrantRecordReporter reporter = mock(ErrantRecordReporter.class); - - // Test both IGNORE and WARN options - // There is no difference in logic between IGNORE and WARN, except for the logging. - // Test to ensure they both work the same logically - final List behaviorsToTest = - Arrays.asList(BehaviorOnMalformedDoc.WARN, BehaviorOnMalformedDoc.IGNORE); - - for(BehaviorOnMalformedDoc behaviorOnMalformedDoc : behaviorsToTest) - { - final String errorInfo = " [{\"type\":\"mapper_parsing_exception\",\"reason\":\"failed to parse\"," + - "\"caused_by\":{\"type\":\"illegal_argument_exception\",\"reason\":\"object\n" + - " field starting or ending with a [.] makes object resolution ambiguous: [avjpz{{.}}wjzse{{..}}gal9d]\"}}]"; - client.expect(Arrays.asList(42, 43), BulkResponse.failure(false, errorInfo, getFailedRecords(42, 43))); - - final BulkProcessor bulkProcessor = new BulkProcessor<>( - Time.SYSTEM, - client, - maxBufferedRecords, - maxInFlightBatches, - batchSize, - lingerMs, - maxRetries, - retryBackoffMs, - behaviorOnMalformedDoc, - reporter - ); - - bulkProcessor.start(); - - bulkProcessor.add(indexableRecord(42), sinkRecord(), 1); - bulkProcessor.add(indexableRecord(43), sinkRecord(), 1); - - try { - final int flushTimeoutMs = 1000; - bulkProcessor.flush(flushTimeoutMs); - } catch (ConnectException e) { - fail(e.getMessage()); - } - assertTrue(bulkProcessor.recordsToReportOnError.isEmpty()); - } - - verify(reporter, times(4)).report(eq(sinkRecord()), any()); - - } - - @Test - public void farmerTaskPropogatesException() { - final int maxBufferedRecords = 100; - final int maxInFlightBatches = 5; - final int batchSize = 2; - final int lingerMs = 5; - final int maxRetries = 3; - final int retryBackoffMs = 1; - final BehaviorOnMalformedDoc behaviorOnMalformedDoc = BehaviorOnMalformedDoc.DEFAULT; - final ErrantRecordReporter reporter = mock(ErrantRecordReporter.class); - - final String errorInfo = "an unretriable error"; - client.expect(Arrays.asList(42, 43), BulkResponse.failure(false, errorInfo, getFailedRecords(42, 43))); - - final BulkProcessor bulkProcessor = new BulkProcessor<>( - Time.SYSTEM, - client, - maxBufferedRecords, - maxInFlightBatches, - batchSize, - lingerMs, - maxRetries, - retryBackoffMs, - behaviorOnMalformedDoc, - reporter - ); - - final int addTimeoutMs = 10; - bulkProcessor.add(indexableRecord(42), sinkRecord(), addTimeoutMs); - bulkProcessor.add(indexableRecord(43), sinkRecord(), addTimeoutMs); - - Runnable farmer = bulkProcessor.farmerTask(); - ConnectException e = assertThrows(ConnectException.class, () -> { - farmer.run(); - // There's a small race condition in the farmer task where a failure on a batch thread - // causes the stopRequested flag of the BulkProcessor to get set, and subsequently checked - // on the farmer task thread, before the batch thread has time to actually complete and - // throw an exception. When this happens, the invocation of farmer::run does not throw an - // exception. However, we can still verify that a batch failed and that the bulk processor - // is in the expected state by invoking BulkProcessor::throwIfFailed, which throws the first - // error encountered on any batch thread. Even if the aforementioned race condition occurs, - // the error should still have been captured by the bulk processor and if it is not present - // by this point, it is a legitimate sign of a bug in the processor instead of a benign race - // condition that just makes testing a little more complicated. - bulkProcessor.throwIfFailed(); - }); - assertThat(e.getMessage(), containsString(errorInfo)); - } - - @Test - public void terminateRetriesWhenInterruptedInSleep() { - final int maxBufferedRecords = 100; - final int maxInFlightBatches = 5; - final int batchSize = 2; - final int lingerMs = 5; - final int maxRetries = 3; - final int retryBackoffMs = 1; - final BehaviorOnMalformedDoc behaviorOnMalformedDoc = BehaviorOnMalformedDoc.DEFAULT; - final ErrantRecordReporter reporter = mock(ErrantRecordReporter.class); - - Time mockTime = mock(Time.class); - doAnswer(invocation -> { - Thread.currentThread().interrupt(); - return null; - }).when(mockTime).sleep(anyLong()); - - client.expect(Arrays.asList(42, 43), BulkResponse.failure(true, "a retriable error", getFailedRecords(42, 43))); - - final BulkProcessor bulkProcessor = new BulkProcessor<>( - mockTime, - client, - maxBufferedRecords, - maxInFlightBatches, - batchSize, - lingerMs, - maxRetries, - retryBackoffMs, - behaviorOnMalformedDoc, - reporter - ); - - final int addTimeoutMs = 10; - bulkProcessor.add(indexableRecord(42), sinkRecord(), addTimeoutMs); - bulkProcessor.add(indexableRecord(43), sinkRecord(), addTimeoutMs); - - ExecutionException e = assertThrows(ExecutionException.class, - () -> bulkProcessor.submitBatchWhenReady().get()); - assertThat(e.getMessage(), containsString("a retriable error")); - assertTrue(bulkProcessor.recordsToReportOnError.isEmpty()); - verify(reporter, never()).report(eq(sinkRecord()), any()); - } - - @Test - public void testNullReporter() { - final int maxBufferedRecords = 100; - final int maxInFlightBatches = 5; - final int batchSize = 2; - final int lingerMs = 5; - final int maxRetries = 3; - final int retryBackoffMs = 1; - final BehaviorOnMalformedDoc behaviorOnMalformedDoc = BehaviorOnMalformedDoc.DEFAULT; - - client.expect(Arrays.asList(42, 43), BulkResponse.success()); - - final BulkProcessor bulkProcessor = new BulkProcessor<>( - Time.SYSTEM, - client, - maxBufferedRecords, - maxInFlightBatches, - batchSize, - lingerMs, - maxRetries, - retryBackoffMs, - behaviorOnMalformedDoc, - null - ); - - bulkProcessor.start(); - - final int addTimeoutMs = 10; - bulkProcessor.add(indexableRecord(42), sinkRecord(), addTimeoutMs); - bulkProcessor.add(indexableRecord(43), sinkRecord(), addTimeoutMs); - - assertEquals(2, bulkProcessor.bufferedRecords()); - assertNull(bulkProcessor.recordsToReportOnError); - - bulkProcessor.flush(1000); - assertEquals(0, bulkProcessor.bufferedRecords()); - } - - @Test - public void reportOnlyFailedRecords() { - final int maxBufferedRecords = 100; - final int maxInFlightBatches = 5; - final int batchSize = 2; - final int lingerMs = 5; - final int maxRetries = 3; - final int retryBackoffMs = 1; - final BehaviorOnMalformedDoc behaviorOnMalformedDoc = BehaviorOnMalformedDoc.IGNORE; - final ErrantRecordReporter reporter = mock(ErrantRecordReporter.class); - - final String errorInfo = " [{\"type\":\"mapper_parsing_exception\",\"reason\":\"failed to parse\"," + - "\"caused_by\":{\"type\":\"illegal_argument_exception\",\"reason\":\"object\n" + - " field starting or ending with a [.] makes object resolution ambiguous: [avjpz{{.}}wjzse{{..}}gal9d]\"}}]"; - client.expect(Arrays.asList(42, 43), BulkResponse.failure(false, errorInfo, getFailedRecords(42))); - - final BulkProcessor bulkProcessor = new BulkProcessor<>( - Time.SYSTEM, - client, - maxBufferedRecords, - maxInFlightBatches, - batchSize, - lingerMs, - maxRetries, - retryBackoffMs, - behaviorOnMalformedDoc, - reporter - ); - - bulkProcessor.start(); - - bulkProcessor.add(indexableRecord(42), sinkRecord(), 1); - bulkProcessor.add(indexableRecord(43), sinkRecord(), 1); - - try { - final int flushTimeoutMs = 1000; - bulkProcessor.flush(flushTimeoutMs); - } catch (ConnectException e) { - fail(e.getMessage()); - } - - assertTrue(bulkProcessor.recordsToReportOnError.isEmpty()); - verify(reporter, times(1)).report(eq(sinkRecord()), any()); - } - - private static IndexableRecord indexableRecord(int id) { - return new IndexableRecord(new Key(INDEX, TYPE, String.valueOf(id)), String.valueOf(id), null); - } - - private static SinkRecord sinkRecord() { - return new SinkRecord(INDEX, 0, Schema.STRING_SCHEMA, "key", Schema.INT32_SCHEMA, 0, 0L); - } - - private static Map getFailedRecords(Integer... ids) { - JestResult jestResult = new JestResult(new Gson()); - JsonObject result = new JsonObject(); - JsonArray array = new JsonArray(); - - for (Integer id : ids) { - JsonObject error = new JsonObject(); - error.addProperty("type", "awful error"); - error.addProperty("reason", "you write bad code"); - - JsonObject values = new JsonObject(); - values.addProperty("_index", INDEX); - values.addProperty("_type", TYPE); - values.addProperty("_id", id.toString()); - values.addProperty("status", 404); - values.addProperty("version", 0); - values.add("error", error); - - JsonObject bulkItemResult = new JsonObject(); - bulkItemResult.add("operation", values); - - array.add(bulkItemResult); - } - - result.add("items", array); - - jestResult.setJsonObject(result); - - Map failedRecords = new HashMap<>(); - BulkResult bulkResult = new BulkResult(jestResult); - List items = bulkResult.getFailedItems(); - for (int i = 0; i < ids.length; i++) { - Integer id = ids[i]; - failedRecords.put(indexableRecord(id), items.get(i)); - } - - return failedRecords; - } -} diff --git a/src/test/java/io/confluent/connect/elasticsearch/integration/ElasticsearchContainer.java b/src/test/java/io/confluent/connect/elasticsearch/helper/ElasticsearchContainer.java similarity index 52% rename from src/test/java/io/confluent/connect/elasticsearch/integration/ElasticsearchContainer.java rename to src/test/java/io/confluent/connect/elasticsearch/helper/ElasticsearchContainer.java index 605ecec92..31e3c86a4 100644 --- a/src/test/java/io/confluent/connect/elasticsearch/integration/ElasticsearchContainer.java +++ b/src/test/java/io/confluent/connect/elasticsearch/helper/ElasticsearchContainer.java @@ -1,21 +1,26 @@ /* - * Copyright [2018 - 2018] Confluent Inc. + * 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.connect.elasticsearch.integration; - -import java.io.File; -import java.io.FileOutputStream; -import java.io.IOException; -import java.io.InputStream; -import java.net.InetAddress; -import java.net.URI; -import java.net.URISyntaxException; -import java.time.Duration; -import java.util.concurrent.Future; +package io.confluent.connect.elasticsearch.helper; +import org.apache.kafka.common.config.SslConfigs; +import org.elasticsearch.client.security.user.User; +import org.elasticsearch.client.security.user.privileges.Role; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.testcontainers.containers.ContainerLaunchException; import org.testcontainers.containers.output.OutputFrame; import org.testcontainers.containers.wait.strategy.Wait; import org.testcontainers.images.RemoteDockerImage; @@ -24,8 +29,26 @@ import org.testcontainers.shaded.org.apache.commons.io.IOUtils; import org.testcontainers.utility.DockerImageName; -import static java.net.HttpURLConnection.HTTP_OK; -import static java.net.HttpURLConnection.HTTP_UNAUTHORIZED; +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.net.InetAddress; +import java.net.URI; +import java.net.URISyntaxException; +import java.time.Duration; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig; +import io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.SecurityProtocol; + +import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.CONNECTION_PASSWORD_CONFIG; +import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.CONNECTION_URL_CONFIG; +import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.CONNECTION_USERNAME_CONFIG; +import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.SECURITY_PROTOCOL_CONFIG; +import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.SSL_CONFIG_PREFIX; /** * A specialized TestContainer container for testing Elasticsearch, optionally with SSL support. @@ -44,7 +67,7 @@ public class ElasticsearchContainer /** * Default Elasticsearch version. */ - public static final String DEFAULT_ES_VERSION = "7.0.0"; + public static final String DEFAULT_ES_VERSION = "7.0.1"; /** * Default Elasticsearch port. @@ -95,13 +118,19 @@ public static ElasticsearchContainer fromSystemProperties() { } private static final String KEY_PASSWORD = "asdfasdf"; - private static final String ELASTIC_PASSWORD = "elastic"; + // Super user that has superuser role. Should not be used by connector + private static final String ELASTIC_SUPERUSER_NAME = "elastic"; + private static final String ELASTIC_SUPERUSER_PASSWORD = "elastic"; + private static final String KEYSTORE_PASSWORD = KEY_PASSWORD; private static final String TRUSTSTORE_PASSWORD = KEY_PASSWORD; private static final long TWO_GIGABYTES = 2L * 1024 * 1024 * 1024; private final String imageName; private boolean enableSsl = false; + private String keytabPath; + private List rolesToCreate; + private Map usersToCreate; private String localKeystorePath; private String localTruststorePath; @@ -117,8 +146,60 @@ public ElasticsearchContainer(String imageName) { withLogConsumer(this::containerLog); } + @Override + public void start() { + super.start(); + String address; + if (isBasicAuthEnabled()) { + Map props = new HashMap<>(); + props.put(CONNECTION_USERNAME_CONFIG, ELASTIC_SUPERUSER_NAME); + props.put(CONNECTION_PASSWORD_CONFIG, ELASTIC_SUPERUSER_PASSWORD); + if (isSslEnabled()) { + addSslProps(props); + address = this.getConnectionUrl(false); + } else { + address = this.getConnectionUrl(); + } + props.put(CONNECTION_URL_CONFIG, address); + ElasticsearchHelperClient helperClient = getHelperClient(props); + createUsersAndRoles(helperClient); + } + } + + public void addSslProps(Map props) { + props.put(SECURITY_PROTOCOL_CONFIG, SecurityProtocol.SSL.name()); + props.put(SSL_CONFIG_PREFIX + SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG, this.getKeystorePath()); + props.put(SSL_CONFIG_PREFIX + SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, this.getKeystorePassword()); + props.put(SSL_CONFIG_PREFIX + SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, this.getTruststorePath()); + props.put(SSL_CONFIG_PREFIX + SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, this.getTruststorePassword()); + props.put(SSL_CONFIG_PREFIX + SslConfigs.SSL_KEY_PASSWORD_CONFIG, this.getKeyPassword()); + } + + private void createUsersAndRoles(ElasticsearchHelperClient helperClient ) { + try { + for (Role role: this.rolesToCreate) { + helperClient.createRole(role); + } + for (Map.Entry userToPassword: this.usersToCreate.entrySet()) { + helperClient.createUser(userToPassword); + } + } catch (IOException e) { + throw new ContainerLaunchException("Container startup failed", e); + } + } + public ElasticsearchContainer withSslEnabled(boolean enable) { - setSslEnabled(enable); + enableSsl(enable); + return this; + } + + public ElasticsearchContainer withKerberosEnabled(String keytab) { + enableKerberos(keytab); + return this; + } + + public ElasticsearchContainer withBasicAuth(Map users, List roles) { + enableBasicAuth(users, roles); return this; } @@ -129,10 +210,10 @@ public ElasticsearchContainer withSslEnabled(boolean enable) { * * @param enable true if SSL is to be enabled, or false otherwise */ - public void setSslEnabled(boolean enable) { + public void enableSsl(boolean enable) { if (isCreated()) { throw new IllegalStateException( - "setSslEnabled can only be used before the Container is created." + "enableSsl can only be used before the Container is created." ); } enableSsl = enable; @@ -147,70 +228,140 @@ public boolean isSslEnabled() { return enableSsl; } + /** + * Set whether the Elasticsearch instance should use Kerberos. + * + *

This can only be called before the container is started. + * + * @param keytab non-null keytab path if Kerberos is enabled + */ + public void enableKerberos(String keytab) { + if (isCreated()) { + throw new IllegalStateException( + "enableKerberos can only be used before the container is created." + ); + } + if (isBasicAuthEnabled()) { + throw new IllegalStateException( + "basic auth and Kerberos are mutually exclusive." + ); + } + keytabPath = keytab; + } + + /** + * Get whether the Elasticsearch instance is configured to use Kerberos. + * + * @return true if Kerberos is enabled, or false otherwise + */ + public boolean isKerberosEnabled() { + return keytabPath != null; + } + + private void enableBasicAuth(Map users, List roles) { + if (isCreated()) { + throw new IllegalStateException( + "enableBasicAuth can only be used before the container is created." + ); + } + if (isKerberosEnabled()) { + throw new IllegalStateException( + "basic auth and Kerberos are mutually exclusive." + ); + } + this.usersToCreate = users; + this.rolesToCreate = roles; + } + + public boolean isBasicAuthEnabled() { + return usersToCreate != null && !this.usersToCreate.isEmpty(); + } + + private String getFullResourcePath(String resourceName) { + if (isSslEnabled() && isKerberosEnabled()) { + return "/both/" + resourceName; + } else if (isSslEnabled()) { + return "/ssl/" + resourceName; + } else if (isKerberosEnabled()) { + return "/kerberos/" + resourceName; + } else if (isBasicAuthEnabled()) { + return "/basic/" + resourceName; + } else { + return resourceName; + } + } + @Override protected void configure() { super.configure(); - Future image; + + waitingFor( + Wait.forLogMessage(".*(Security is enabled|license .* valid).*", 1) + .withStartupTimeout(Duration.ofMinutes(5)) + ); + + if (!isSslEnabled() && !isKerberosEnabled() && !isBasicAuthEnabled()) { + setImage(new RemoteDockerImage(DockerImageName.parse(imageName))); + return; + } + + ImageFromDockerfile image = new ImageFromDockerfile() + // Copy the Elasticsearch config file + .withFileFromClasspath("elasticsearch.yml", getFullResourcePath("elasticsearch.yml")) + // Copy the network definitions + .withFileFromClasspath("instances.yml", getFullResourcePath("instances.yml")) + .withDockerfileFromBuilder(this::buildImage); + + // Kerberos and basic auth are mutually exclusive authentication options + if (isBasicAuthEnabled()) { + log.info("Setting up basic authentication in a Docker image"); + withEnv("ELASTICSEARCH_USERNAME", ELASTIC_SUPERUSER_NAME); + withEnv("ELASTIC_PASSWORD", ELASTIC_SUPERUSER_PASSWORD); + } else if (isKerberosEnabled()) { + log.info("Creating Kerberized Elasticsearch image."); + image.withFileFromFile("es.keytab", new File(keytabPath)); + } if (isSslEnabled()) { - withEnv("ELASTIC_PASSWORD", ELASTIC_PASSWORD); + log.info("Extending Docker image to generate certs and enable SSL"); + withEnv("ELASTIC_PASSWORD", ELASTIC_SUPERUSER_PASSWORD); withEnv("STORE_PASSWORD", KEY_PASSWORD); withEnv("IP_ADDRESS", hostMachineIpAddress()); - log.info("Extending Docker image to generate certs and enable SSL"); - log.info("Wait for 'license .* valid' in log file, signaling Elasticsearch has started"); - // Because this is an secured Elasticsearch instance, we can't use HTTPS checks - // because of the untrusted cert - waitingFor( - Wait.forLogMessage(".*(Security is enabled|license .* valid).*", 1) - .withStartupTimeout(Duration.ofMinutes(5)) - ); - image = new ImageFromDockerfile() - // Copy the Elasticsearch config file for SSL - .withFileFromClasspath( - "elasticsearch.yml", - "/ssl/elasticsearch.yml" - ) - // Copy the network definitions - .withFileFromClasspath( - "instances.yml", - "/ssl/instances.yml" - ) + + image // Copy the script to generate the certs and start Elasticsearch - .withFileFromClasspath( - "start-elasticsearch.sh", - "/ssl/start-elasticsearch.sh" - ) - .withDockerfileFromBuilder(this::build); - } else { - log.info("Will use HTTP check to wait for Elasticsearch image"); - // Because this is an unsecured Elasticsearch instance, we can use HTTP checks - waitingFor( - Wait.forHttp("/") - .forPort(ELASTICSEARCH_DEFAULT_PORT) - .forStatusCodeMatching(status -> status == HTTP_OK || status == HTTP_UNAUTHORIZED) - .withStartupTimeout(Duration.ofMinutes(2)) - ); - image = new RemoteDockerImage(DockerImageName.parse(imageName)); + .withFileFromClasspath("start-elasticsearch.sh", + getFullResourcePath("start-elasticsearch.sh")); } setImage(image); } - protected void build(DockerfileBuilder builder) { - log.info("Building Elasticsearch image with SSL configuration"); - builder.from(imageName) - // OpenSSL and Java's Keytool used to generate the certs, so install them - .run("yum -y install openssl") - // Copy the Elasticsearch configuration - .copy("elasticsearch.yml", CONFIG_PATH +"/elasticsearch.yml") - // Copy and run the script to generate the certs - .copy("instances.yml", CONFIG_SSL_PATH + "/instances.yml") - .copy("start-elasticsearch.sh", CONFIG_SSL_PATH + "/start-elasticsearch.sh") - .run("chmod +x " + CONFIG_SSL_PATH + "/start-elasticsearch.sh") - .entryPoint( - CONFIG_SSL_PATH + "/start-elasticsearch.sh" - ); + private void buildImage(DockerfileBuilder builder) { + builder + .from(imageName) + // Copy the Elasticsearch configuration + .copy("elasticsearch.yml", CONFIG_PATH + "/elasticsearch.yml"); + + if (isSslEnabled()) { + log.info("Building Elasticsearch image with SSL configuration"); + builder + .copy("instances.yml", CONFIG_SSL_PATH + "/instances.yml") + .copy("start-elasticsearch.sh", CONFIG_SSL_PATH + "/start-elasticsearch.sh") + // OpenSSL and Java's Keytool used to generate the certs, so install them + .run("yum -y install openssl") + .run("chmod +x " + CONFIG_SSL_PATH + "/start-elasticsearch.sh") + .entryPoint(CONFIG_SSL_PATH + "/start-elasticsearch.sh"); + } + + if (isKerberosEnabled()) { + log.info("Building Elasticsearch image with Kerberos configuration."); + builder.copy("es.keytab", CONFIG_PATH + "/es.keytab"); + if (!isSslEnabled()) { + builder.copy("instances.yml", CONFIG_PATH + "/instances.yml"); + } + } } - protected String hostMachineIpAddress() { + public String hostMachineIpAddress() { String dockerHost = System.getenv("DOCKER_HOST"); if (dockerHost != null && !dockerHost.trim().isEmpty()) { try { @@ -231,19 +382,28 @@ protected String hostMachineIpAddress() { } } + /** + * @see ElasticsearchContainer#getConnectionUrl(boolean) + */ + public String getConnectionUrl() { + return getConnectionUrl(true); + } + /** * Get the Elasticsearch connection URL. * *

This can only be called once the container is started. * + * @param useContainerIpAddress use container IP if true, host machine's IP otherwise + * * @return the connection URL; never null */ - public String getConnectionUrl() { + public String getConnectionUrl(boolean useContainerIpAddress) { String protocol = isSslEnabled() ? "https" : "http"; return String.format( "%s://%s:%d", protocol, - getContainerIpAddress(), + useContainerIpAddress ? getContainerIpAddress() : hostMachineIpAddress(), getMappedPort(ELASTICSEARCH_DEFAULT_PORT) ); } @@ -377,4 +537,14 @@ protected void containerLog(OutputFrame logMessage) { break; } } + + public ElasticsearchHelperClient getHelperClient(Map props) { + // copy properties so that original properties are not affected + Map superUserProps = new HashMap<>(props); + superUserProps.put(CONNECTION_USERNAME_CONFIG, ELASTIC_SUPERUSER_NAME); + superUserProps.put(CONNECTION_PASSWORD_CONFIG, ELASTIC_SUPERUSER_PASSWORD); + ElasticsearchSinkConnectorConfig config = new ElasticsearchSinkConnectorConfig(superUserProps); + ElasticsearchHelperClient client = new ElasticsearchHelperClient(props.get(CONNECTION_URL_CONFIG), config); + return client; + } } diff --git a/src/test/java/io/confluent/connect/elasticsearch/helper/ElasticsearchHelperClient.java b/src/test/java/io/confluent/connect/elasticsearch/helper/ElasticsearchHelperClient.java new file mode 100644 index 000000000..5276c59e4 --- /dev/null +++ b/src/test/java/io/confluent/connect/elasticsearch/helper/ElasticsearchHelperClient.java @@ -0,0 +1,115 @@ +/* + * Copyright 2020 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.connect.elasticsearch.helper; + +import org.apache.http.HttpHost; +import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequest; +import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.client.RequestOptions; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.RestHighLevelClient; +import org.elasticsearch.client.core.CountRequest; +import org.elasticsearch.client.indices.GetIndexRequest; +import org.elasticsearch.client.indices.GetMappingsRequest; +import org.elasticsearch.client.indices.GetMappingsResponse; +import org.elasticsearch.client.security.PutRoleRequest; +import org.elasticsearch.client.security.PutRoleResponse; +import org.elasticsearch.client.security.PutUserRequest; +import org.elasticsearch.client.security.PutUserResponse; +import org.elasticsearch.client.security.RefreshPolicy; +import org.elasticsearch.client.security.user.User; +import org.elasticsearch.client.security.user.privileges.Role; +import org.elasticsearch.cluster.metadata.MappingMetadata; +import org.elasticsearch.search.SearchHits; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Map.Entry; + +import io.confluent.connect.elasticsearch.ConfigCallbackHandler; +import io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig; + +public class ElasticsearchHelperClient { + + private static final Logger log = LoggerFactory.getLogger(ElasticsearchHelperClient.class); + + private RestHighLevelClient client; + + public ElasticsearchHelperClient(String url, ElasticsearchSinkConnectorConfig config) { + ConfigCallbackHandler configCallbackHandler = new ConfigCallbackHandler(config); + this.client = new RestHighLevelClient( + RestClient + .builder(HttpHost.create(url)) + .setHttpClientConfigCallback(configCallbackHandler) + ); + } + + public void deleteIndex(String index) throws IOException { + DeleteIndexRequest request = new DeleteIndexRequest(index); + client.indices().delete(request, RequestOptions.DEFAULT); + } + + public long getDocCount(String index) throws IOException { + CountRequest request = new CountRequest(index); + return client.count(request, RequestOptions.DEFAULT).getCount(); + } + + public MappingMetadata getMapping(String index) throws IOException { + GetMappingsRequest request = new GetMappingsRequest().indices(index); + GetMappingsResponse response = client.indices().getMapping(request, RequestOptions.DEFAULT); + return response.mappings().get(index); + } + + public boolean indexExists(String index) throws IOException { + GetIndexRequest request = new GetIndexRequest(index); + return client.indices().exists(request, RequestOptions.DEFAULT); + } + + public SearchHits search(String index) throws IOException { + SearchRequest request = new SearchRequest(index); + return client.search(request, RequestOptions.DEFAULT).getHits(); + } + + public void createRole(Role role) throws IOException { + PutRoleRequest putRoleRequest = new PutRoleRequest(role, RefreshPolicy.IMMEDIATE); + PutRoleResponse putRoleResponse = client.security().putRole(putRoleRequest, RequestOptions.DEFAULT); + if (!putRoleResponse.isCreated()) { + throw new RuntimeException(String.format("Failed to create a role %s", role.getName())); + } + } + + public void createUser(Entry userToPassword) throws IOException { + PutUserRequest putUserRequest = PutUserRequest.withPassword( + userToPassword.getKey(), + userToPassword.getValue().toCharArray(), + true, + RefreshPolicy.IMMEDIATE + ); + PutUserResponse putUserResponse = client.security().putUser(putUserRequest, RequestOptions.DEFAULT); + if (!putUserResponse.isCreated()) { + throw new RuntimeException(String.format("Failed to create a user %s", userToPassword.getKey().getUsername())); + } + } + + public void close() { + try { + client.close(); + } catch (IOException e) { + log.error("Error closing client.", e); + } + } +} diff --git a/src/test/java/io/confluent/connect/elasticsearch/helper/NetworkErrorContainer.java b/src/test/java/io/confluent/connect/elasticsearch/helper/NetworkErrorContainer.java new file mode 100644 index 000000000..43f847730 --- /dev/null +++ b/src/test/java/io/confluent/connect/elasticsearch/helper/NetworkErrorContainer.java @@ -0,0 +1,44 @@ +/* + * 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.connect.elasticsearch.helper; + +import org.testcontainers.containers.BindMode; +import org.testcontainers.containers.GenericContainer; +import org.testcontainers.containers.wait.strategy.Wait; + +public class NetworkErrorContainer extends GenericContainer { + + private static final String DEFAULT_DOCKER_IMAGE = "gaiaadm/pumba:latest"; + + private static final String PUMBA_PAUSE_COMMAND = "--log-level info --interval 120s pause --duration 10s "; + private static final String DOCKER_SOCK = "/var/run/docker.sock"; + + public NetworkErrorContainer(String containerToInterrupt) { + this(DEFAULT_DOCKER_IMAGE, containerToInterrupt); + } + + public NetworkErrorContainer( + String dockerImageName, + String containerToInterrupt + ) { + super(dockerImageName); + + setCommand(PUMBA_PAUSE_COMMAND + containerToInterrupt); + addFileSystemBind(DOCKER_SOCK, DOCKER_SOCK, BindMode.READ_WRITE); + setWaitStrategy(Wait.forLogMessage(".*pausing container.*", 1)); + withLogConsumer(l -> System.out.print(l.getUtf8String())); + } +} \ No newline at end of file diff --git a/src/test/java/io/confluent/connect/elasticsearch/integration/ElasticsearchConnectorBaseIT.java b/src/test/java/io/confluent/connect/elasticsearch/integration/ElasticsearchConnectorBaseIT.java index af71d6d31..e0edc3182 100644 --- a/src/test/java/io/confluent/connect/elasticsearch/integration/ElasticsearchConnectorBaseIT.java +++ b/src/test/java/io/confluent/connect/elasticsearch/integration/ElasticsearchConnectorBaseIT.java @@ -15,10 +15,31 @@ package io.confluent.connect.elasticsearch.integration; +import io.confluent.connect.elasticsearch.ElasticsearchSinkConnector; +import io.confluent.connect.elasticsearch.helper.ElasticsearchContainer; +import io.confluent.connect.elasticsearch.helper.ElasticsearchHelperClient; +import org.apache.kafka.connect.json.JsonConverter; +import org.apache.kafka.connect.storage.StringConverter; +import org.apache.kafka.test.TestUtils; +import org.elasticsearch.ElasticsearchStatusException; +import org.elasticsearch.client.security.user.User; +import org.elasticsearch.client.security.user.privileges.IndicesPrivileges; +import org.elasticsearch.client.security.user.privileges.Role; +import org.elasticsearch.client.security.user.privileges.Role.Builder; +import org.elasticsearch.search.SearchHit; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; + +import java.net.ConnectException; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.CONNECTION_URL_CONFIG; import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.IGNORE_KEY_CONFIG; import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.IGNORE_SCHEMA_CONFIG; -import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.TYPE_NAME_CONFIG; import static org.apache.kafka.connect.json.JsonConverterConfig.SCHEMAS_ENABLE_CONFIG; import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLASS_CONFIG; import static org.apache.kafka.connect.runtime.ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG; @@ -26,39 +47,29 @@ import static org.apache.kafka.connect.runtime.ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG; import static org.apache.kafka.connect.runtime.SinkConnectorConfig.TOPICS_CONFIG; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; -import com.google.gson.JsonArray; -import com.google.gson.JsonObject; -import io.confluent.connect.elasticsearch.ElasticsearchClient; -import io.confluent.connect.elasticsearch.ElasticsearchSinkConnector; -import io.confluent.connect.elasticsearch.jest.JestElasticsearchClient; -import java.io.IOException; -import java.util.HashMap; -import java.util.Map; -import org.apache.kafka.connect.json.JsonConverter; -import org.apache.kafka.connect.storage.StringConverter; -import org.apache.kafka.test.TestUtils; -import org.junit.After; -import org.junit.AfterClass; -import org.junit.Before; -import org.junit.BeforeClass; - public class ElasticsearchConnectorBaseIT extends BaseConnectorIT { protected static final int NUM_RECORDS = 5; protected static final int TASKS_MAX = 1; protected static final String CONNECTOR_NAME = "es-connector"; protected static final String TOPIC = "test"; - protected static final String TYPE = "kafka-connect"; + + // User that has a minimal required and documented set of privileges + public static final String ELASTIC_MINIMAL_PRIVILEGES_NAME = "frank"; + public static final String ELASTIC_MINIMAL_PRIVILEGES_PASSWORD = "WatermelonInEasterHay"; + + private static final String ES_SINK_CONNECTOR_ROLE = "es_sink_connector_role"; protected static ElasticsearchContainer container; - protected ElasticsearchClient client; + protected ElasticsearchHelperClient helperClient; protected Map props; @BeforeClass - public static void setupBeforeAll() { + public static void setupBeforeAll() throws Exception { container = ElasticsearchContainer.fromSystemProperties(); container.start(); } @@ -74,18 +85,23 @@ public void setup() throws Exception { connect.kafka().createTopic(TOPIC); props = createProps(); - client = createClient(); + helperClient = container.getHelperClient(props); } @After public void cleanup() throws Exception { stopConnect(); - client.deleteAll(); - client.close(); - } - protected ElasticsearchClient createClient() { - return new JestElasticsearchClient(container.getConnectionUrl()); + if (container.isRunning()) { + if (helperClient != null) { + try { + helperClient.deleteIndex(TOPIC); + helperClient.close(); + } catch (ConnectException e) { + // Server is already down. No need to close + } + } + } } protected Map createProps() { @@ -100,7 +116,6 @@ protected Map createProps() { props.put("value.converter." + SCHEMAS_ENABLE_CONFIG, "false"); // connectors specific - props.put(TYPE_NAME_CONFIG, TYPE); props.put(CONNECTION_URL_CONFIG, container.getConnectionUrl()); props.put(IGNORE_KEY_CONFIG, "true"); props.put(IGNORE_SCHEMA_CONFIG, "true"); @@ -119,29 +134,15 @@ protected void runSimpleTest(Map props) throws Exception { verifySearchResults(NUM_RECORDS); } - protected void writeRecords(int numRecords) { - writeRecordsFromIndex(0, numRecords); - } - - protected void writeRecordsFromIndex(int start, int numRecords) { - for (int i = start; i < start + numRecords; i++) { - connect.kafka().produce(TOPIC, String.valueOf(i), String.format("{\"doc_num\":%d}", i)); - } - } protected void verifySearchResults(int numRecords) throws Exception { waitForRecords(numRecords); - final JsonObject result = client.search("", TOPIC, null); - final JsonArray rawHits = result.getAsJsonObject("hits").getAsJsonArray("hits"); - - assertEquals(numRecords, rawHits.size()); - - for (int i = 0; i < rawHits.size(); ++i) { - final JsonObject hitData = rawHits.get(i).getAsJsonObject(); - final JsonObject source = hitData.get("_source").getAsJsonObject(); - assertTrue(source.has("doc_num")); - assertTrue(source.get("doc_num").getAsInt() < numRecords); + for (SearchHit hit : helperClient.search(TOPIC)) { + int id = (Integer) hit.getSourceAsMap().get("doc_num"); + assertNotNull(id); + assertTrue(id < numRecords); + assertEquals(TOPIC, hit.getIndex()); } } @@ -149,13 +150,13 @@ protected void waitForRecords(int numRecords) throws InterruptedException { TestUtils.waitForCondition( () -> { try { - client.refresh(); - final JsonObject result = client.search("", TOPIC, null); - return result.getAsJsonObject("hits") != null - && result.getAsJsonObject("hits").getAsJsonArray("hits").size() == numRecords; + return helperClient.getDocCount(TOPIC) == numRecords; + } catch (ElasticsearchStatusException e) { + if (e.getMessage().contains("index_not_found_exception")) { + return false; + } - } catch (IOException e) { - return false; + throw e; } }, CONSUME_MAX_DURATION_MS, @@ -163,4 +164,35 @@ protected void waitForRecords(int numRecords) throws InterruptedException { ); } + protected void writeRecords(int numRecords) { + for (int i = 0; i < numRecords; i++) { + connect.kafka().produce(TOPIC, String.valueOf(i), String.format("{\"doc_num\":%d}", i)); + } + } + + protected void writeRecordsFromStartIndex(int start, int numRecords) { + for (int i = start; i < start + numRecords; i++) { + connect.kafka().produce(TOPIC, String.valueOf(i), String.format("{\"doc_num\":%d}", i)); + } + } + + protected static Role getMinimalPrivilegesRole() { + IndicesPrivileges.Builder indicesPrivilegesBuilder = IndicesPrivileges.builder(); + IndicesPrivileges indicesPrivileges = indicesPrivilegesBuilder + .indices("*") + .privileges("create_index", "read", "write", "view_index_metadata") + .build(); + Builder builder = Role.builder(); + Role role = builder.name(ES_SINK_CONNECTOR_ROLE).indicesPrivileges(indicesPrivileges).build(); + return role; + } + + protected static User getMinimalPrivilegesUser() { + return new User(ELASTIC_MINIMAL_PRIVILEGES_NAME, + Collections.singletonList(ES_SINK_CONNECTOR_ROLE)); + } + + protected static String getMinimalPrivilegesPassword() { + return ELASTIC_MINIMAL_PRIVILEGES_PASSWORD; + } } diff --git a/src/test/java/io/confluent/connect/elasticsearch/integration/ElasticsearchConnectorDataFormatIT.java b/src/test/java/io/confluent/connect/elasticsearch/integration/ElasticsearchConnectorDataFormatIT.java index 4e002550b..59e15418a 100644 --- a/src/test/java/io/confluent/connect/elasticsearch/integration/ElasticsearchConnectorDataFormatIT.java +++ b/src/test/java/io/confluent/connect/elasticsearch/integration/ElasticsearchConnectorDataFormatIT.java @@ -76,21 +76,14 @@ protected void waitForSchemaRegistryToStart() throws InterruptedException { @Override public void setup() throws Exception { - startConnect(); + super.setup(); startSchemaRegistry(); - connect.kafka().createTopic(TOPIC); - - props = createProps(); - client = createClient(); } @Override public void cleanup() throws Exception { - stopConnect(); - stopSchemaRegistry(); - client.deleteAll(); - client.close(); - } + super.cleanup(); + stopSchemaRegistry();} @Parameters public static List> data() { diff --git a/src/test/java/io/confluent/connect/elasticsearch/integration/ElasticsearchConnectorIT.java b/src/test/java/io/confluent/connect/elasticsearch/integration/ElasticsearchConnectorIT.java index 37d5b7e25..b07e793bf 100644 --- a/src/test/java/io/confluent/connect/elasticsearch/integration/ElasticsearchConnectorIT.java +++ b/src/test/java/io/confluent/connect/elasticsearch/integration/ElasticsearchConnectorIT.java @@ -17,34 +17,89 @@ import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.BATCH_SIZE_CONFIG; import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.BEHAVIOR_ON_NULL_VALUES_CONFIG; +import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.CONNECTION_PASSWORD_CONFIG; +import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.CONNECTION_USERNAME_CONFIG; import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.IGNORE_KEY_CONFIG; import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.LINGER_MS_CONFIG; import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.WRITE_METHOD_CONFIG; import static org.apache.kafka.connect.runtime.ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; +import static org.assertj.core.api.Assertions.assertThat; +import static org.awaitility.Awaitility.await; +import static org.junit.Assert.assertEquals; -import com.google.gson.JsonArray; -import com.google.gson.JsonObject; -import io.confluent.connect.elasticsearch.DataConverter.BehaviorOnNullValues; -import io.confluent.connect.elasticsearch.Mapping; -import io.confluent.connect.elasticsearch.TestUtils; -import io.confluent.connect.elasticsearch.jest.JestElasticsearchClient.WriteMethod; +import io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig; +import io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.BehaviorOnNullValues; +import io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.WriteMethod; +import io.confluent.connect.elasticsearch.helper.ElasticsearchContainer; -import java.util.Collections; - -import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.storage.StringConverter; -import org.apache.kafka.test.IntegrationTest; +import io.confluent.common.utils.IntegrationTest; +import org.elasticsearch.client.security.user.User; +import org.elasticsearch.client.security.user.privileges.Role; +import org.elasticsearch.search.SearchHit; +import org.junit.BeforeClass; import org.junit.Test; import org.junit.experimental.categories.Category; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; + +import java.time.Duration; +import java.util.Collections; +import java.util.List; +import java.util.Map; @Category(IntegrationTest.class) public class ElasticsearchConnectorIT extends ElasticsearchConnectorBaseIT { - private static Logger log = LoggerFactory.getLogger(ElasticsearchConnectorIT.class); + // TODO: test compatibility + + @BeforeClass + public static void setupBeforeAll() { + Map users = Collections.singletonMap(getMinimalPrivilegesUser(), getMinimalPrivilegesPassword()); + List roles = Collections.singletonList(getMinimalPrivilegesRole()); + container = ElasticsearchContainer.fromSystemProperties().withBasicAuth(users, roles); + container.start(); + } + + @Override + public void setup() throws Exception { + if (!container.isRunning()) { + setupBeforeAll(); + } + super.setup(); + } + + @Override + protected Map createProps() { + props = super.createProps(); + props.put(CONNECTION_USERNAME_CONFIG, ELASTIC_MINIMAL_PRIVILEGES_NAME); + props.put(CONNECTION_PASSWORD_CONFIG, ELASTIC_MINIMAL_PRIVILEGES_PASSWORD); + return props; + } + + @Test + public void testStopESContainer() throws Exception { + props.put(ElasticsearchSinkConnectorConfig.MAX_RETRIES_CONFIG, "2"); + props.put(ElasticsearchSinkConnectorConfig.RETRY_BACKOFF_MS_CONFIG, "10"); + props.put(ElasticsearchSinkConnectorConfig.BATCH_SIZE_CONFIG, "1"); + props.put(ElasticsearchSinkConnectorConfig.MAX_IN_FLIGHT_REQUESTS_CONFIG, + Integer.toString(NUM_RECORDS - 1)); + + // run connector and write + runSimpleTest(props); + + // stop ES, for all following requests to fail with "connection refused" + container.stop(); + + // try to write some more + writeRecords(NUM_RECORDS); + + // Connector should fail since the server is down + await().atMost(Duration.ofMinutes(1)).untilAsserted(() -> + assertThat(connect.connectorStatus(CONNECTOR_NAME).tasks().get(0).state()) + .isEqualTo("FAILED")); + + assertThat(connect.connectorStatus(CONNECTOR_NAME).tasks().get(0).trace()) + .contains("'java.net.ConnectException: Connection refused' after 3 attempt(s)"); + } @Test public void testChangeConfigsAndRestart() throws Exception { @@ -81,18 +136,6 @@ public void testHappyPath() throws Exception { runSimpleTest(props); } - @Test - @SuppressWarnings("unchecked") - public void testMapping() throws Exception { - client.createIndices(Collections.singleton(TOPIC)); - Schema schema = TestUtils.createSchema(); - Mapping.createMapping(client, TOPIC, TYPE, schema); - - JsonObject mapping = Mapping.getMapping(client, TOPIC, TYPE); - assertNotNull(mapping); - TestUtils.verifyMapping(client, schema, mapping); - } - @Test public void testNullValue() throws Exception { runSimpleTest(props); @@ -135,21 +178,16 @@ public void testUpsert() throws Exception { // try updating last one int lastRecord = NUM_RECORDS - 1; connect.kafka().produce(TOPIC, String.valueOf(lastRecord), String.format("{\"doc_num\":%d}", 0)); - writeRecordsFromIndex(NUM_RECORDS, NUM_RECORDS); + writeRecordsFromStartIndex(NUM_RECORDS, NUM_RECORDS); // should have double number of records verifySearchResults(NUM_RECORDS * 2); - JsonObject result = client.search("", TOPIC, null); - JsonArray rawHits = result.getAsJsonObject("hits").getAsJsonArray("hits"); - - for (int i = 0; i < rawHits.size(); ++i) { - JsonObject hitData = rawHits.get(i).getAsJsonObject(); - JsonObject source = hitData.get("_source").getAsJsonObject(); - assertTrue(source.has("doc_num")); - if (Integer.valueOf(hitData.get("_id").getAsString()) == lastRecord) { + for (SearchHit hit : helperClient.search(TOPIC)) { + if (Integer.parseInt(hit.getId()) == lastRecord) { // last record should be updated - assertTrue(source.get("doc_num").getAsInt() == 0); + int docNum = (Integer) hit.getSourceAsMap().get("doc_num"); + assertEquals(0, docNum); } } } diff --git a/src/test/java/io/confluent/connect/elasticsearch/integration/ElasticsearchConnectorKerberosIT.java b/src/test/java/io/confluent/connect/elasticsearch/integration/ElasticsearchConnectorKerberosIT.java new file mode 100644 index 000000000..c6881f0dc --- /dev/null +++ b/src/test/java/io/confluent/connect/elasticsearch/integration/ElasticsearchConnectorKerberosIT.java @@ -0,0 +1,104 @@ +package io.confluent.connect.elasticsearch.integration; + +import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.KERBEROS_KEYTAB_PATH_CONFIG; +import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.KERBEROS_PRINCIPAL_CONFIG; + +import io.confluent.connect.elasticsearch.helper.ElasticsearchContainer; +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.Comparator; +import java.util.Map; +import java.util.Properties; +import org.apache.hadoop.minikdc.MiniKdc; +import org.apache.kafka.connect.errors.ConnectException; +import io.confluent.common.utils.IntegrationTest; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category(IntegrationTest.class) +public class ElasticsearchConnectorKerberosIT extends ElasticsearchConnectorBaseIT { + + private static File baseDir; + private static MiniKdc kdc; + private static String esPrincipal; + protected static String esKeytab; + private static String userPrincipal; + private static String userKeytab; + + @BeforeClass + public static void setupBeforeAll() throws Exception { + initKdc(); + + container = ElasticsearchContainer.fromSystemProperties().withKerberosEnabled(esKeytab); + container.start(); + } + + /** + * Shuts down the KDC and cleans up files. + */ + @AfterClass + public static void cleanupAfterAll() { + container.close(); + closeKdc(); + } + + @Test + public void testKerberos() throws Exception { + addKerberosConfigs(props); + helperClient = container.getHelperClient(props); + runSimpleTest(props); + } + + protected static void initKdc() throws Exception { + baseDir = new File(System.getProperty("test.build.dir", "target/test-dir")); + if (baseDir.exists()) { + deleteDirectory(baseDir.toPath()); + } + + Properties kdcConf = MiniKdc.createConf(); + kdc = new MiniKdc(kdcConf, baseDir); + kdc.start(); + + String es = "es"; + File keytabFile = new File(baseDir, es + ".keytab"); + esKeytab = keytabFile.getAbsolutePath(); + kdc.createPrincipal(keytabFile, es + "/localhost", "HTTP/localhost"); + esPrincipal = es + "/localhost@" + kdc.getRealm(); + + String user = "connect-es"; + keytabFile = new File(baseDir, user + ".keytab"); + userKeytab = keytabFile.getAbsolutePath(); + kdc.createPrincipal(keytabFile, user + "/localhost"); + userPrincipal = user + "/localhost@" + kdc.getRealm(); + } + + protected static void addKerberosConfigs(Map props) { + props.put(KERBEROS_PRINCIPAL_CONFIG, userPrincipal); + props.put(KERBEROS_KEYTAB_PATH_CONFIG, userKeytab); + } + + private static void closeKdc() { + if (kdc != null) { + kdc.stop(); + } + + if (baseDir.exists()) { + deleteDirectory(baseDir.toPath()); + } + } + + private static void deleteDirectory(Path directoryPath) { + try { + Files.walk(directoryPath) + .sorted(Comparator.reverseOrder()) + .map(Path::toFile) + .forEach(File::delete); + } catch (IOException e) { + throw new ConnectException(e); + } + } +} diff --git a/src/test/java/io/confluent/connect/elasticsearch/integration/ElasticsearchConnectorKerberosWithSslIT.java b/src/test/java/io/confluent/connect/elasticsearch/integration/ElasticsearchConnectorKerberosWithSslIT.java new file mode 100644 index 000000000..2aad4d911 --- /dev/null +++ b/src/test/java/io/confluent/connect/elasticsearch/integration/ElasticsearchConnectorKerberosWithSslIT.java @@ -0,0 +1,57 @@ +package io.confluent.connect.elasticsearch.integration; + +import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.CONNECTION_URL_CONFIG; +import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.SECURITY_PROTOCOL_CONFIG; +import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.SSL_CONFIG_PREFIX; + +import io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.SecurityProtocol; +import io.confluent.connect.elasticsearch.helper.ElasticsearchContainer; +import org.apache.kafka.common.config.SslConfigs; +import io.confluent.common.utils.IntegrationTest; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.Ignore; + +@Category(IntegrationTest.class) +public class ElasticsearchConnectorKerberosWithSslIT extends ElasticsearchConnectorKerberosIT{ + + @BeforeClass + public static void setupBeforeAll() throws Exception { + initKdc(); + + container = ElasticsearchContainer + .fromSystemProperties() + .withKerberosEnabled(esKeytab) + .withSslEnabled(true); + container.start(); + } + + @Override + @Test + public void testKerberos() { + // skip as parent is running this + helperClient = null; + } + + @Ignore("flaky") + @Test + public void testKerberosWithSsl() throws Exception { + // Use IP address here because that's what the certificates allow + String address = container.getConnectionUrl(false); + + props.put(CONNECTION_URL_CONFIG, address); + props.put(SECURITY_PROTOCOL_CONFIG, SecurityProtocol.SSL.name()); + props.put(SSL_CONFIG_PREFIX + SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG, container.getKeystorePath()); + props.put(SSL_CONFIG_PREFIX + SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, container.getKeystorePassword()); + props.put(SSL_CONFIG_PREFIX + SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, container.getTruststorePath()); + props.put(SSL_CONFIG_PREFIX + SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, container.getTruststorePassword()); + props.put(SSL_CONFIG_PREFIX + SslConfigs.SSL_KEY_PASSWORD_CONFIG, container.getKeyPassword()); + addKerberosConfigs(props); + + helperClient = container.getHelperClient(props); + + // Start connector + runSimpleTest(props); + } +} diff --git a/src/test/java/io/confluent/connect/elasticsearch/integration/ElasticsearchConnectorNetworkIT.java b/src/test/java/io/confluent/connect/elasticsearch/integration/ElasticsearchConnectorNetworkIT.java new file mode 100644 index 000000000..adc984c8b --- /dev/null +++ b/src/test/java/io/confluent/connect/elasticsearch/integration/ElasticsearchConnectorNetworkIT.java @@ -0,0 +1,339 @@ +package io.confluent.connect.elasticsearch.integration; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ObjectNode; +import com.github.tomakehurst.wiremock.WireMockServer; +import com.github.tomakehurst.wiremock.client.WireMock; +import com.github.tomakehurst.wiremock.common.FileSource; +import com.github.tomakehurst.wiremock.extension.Parameters; +import com.github.tomakehurst.wiremock.extension.ResponseTransformer; +import com.github.tomakehurst.wiremock.http.Request; +import com.github.tomakehurst.wiremock.http.Response; +import com.github.tomakehurst.wiremock.junit.WireMockRule; +import com.github.tomakehurst.wiremock.stubbing.Scenario; +import io.confluent.connect.elasticsearch.ElasticsearchSinkConnector; +import org.apache.kafka.connect.errors.ConnectException; +import org.apache.kafka.connect.json.JsonConverter; +import org.apache.kafka.connect.storage.StringConverter; +import org.junit.After; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.Ignore; + +import java.time.Duration; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.Semaphore; +import java.util.concurrent.atomic.AtomicInteger; + +import static com.github.tomakehurst.wiremock.client.WireMock.aResponse; +import static com.github.tomakehurst.wiremock.client.WireMock.any; +import static com.github.tomakehurst.wiremock.client.WireMock.anyUrl; +import static com.github.tomakehurst.wiremock.client.WireMock.ok; +import static com.github.tomakehurst.wiremock.client.WireMock.okJson; +import static com.github.tomakehurst.wiremock.client.WireMock.post; +import static com.github.tomakehurst.wiremock.client.WireMock.postRequestedFor; +import static com.github.tomakehurst.wiremock.client.WireMock.stubFor; +import static com.github.tomakehurst.wiremock.client.WireMock.urlPathEqualTo; +import static com.github.tomakehurst.wiremock.client.WireMock.verify; +import static com.github.tomakehurst.wiremock.core.WireMockConfiguration.options; +import static com.google.common.net.HttpHeaders.CONTENT_TYPE; +import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.BATCH_SIZE_CONFIG; +import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.CONNECTION_URL_CONFIG; +import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.IGNORE_KEY_CONFIG; +import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.IGNORE_SCHEMA_CONFIG; +import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.LINGER_MS_CONFIG; +import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.MAX_IN_FLIGHT_REQUESTS_CONFIG; +import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.MAX_RETRIES_CONFIG; +import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.READ_TIMEOUT_MS_CONFIG; +import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.RETRY_BACKOFF_MS_CONFIG; +import static org.apache.kafka.connect.json.JsonConverterConfig.SCHEMAS_ENABLE_CONFIG; +import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLASS_CONFIG; +import static org.apache.kafka.connect.runtime.ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG; +import static org.apache.kafka.connect.runtime.ConnectorConfig.TASKS_MAX_CONFIG; +import static org.apache.kafka.connect.runtime.ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG; +import static org.apache.kafka.connect.runtime.SinkConnectorConfig.TOPICS_CONFIG; +import static org.assertj.core.api.Assertions.assertThat; +import static org.awaitility.Awaitility.await; + +public class ElasticsearchConnectorNetworkIT extends BaseConnectorIT { + + @Rule + public WireMockRule wireMockRule = new WireMockRule(options().dynamicPort(), false); + + private static final ObjectMapper MAPPER = new ObjectMapper(); + + private static final int NUM_RECORDS = 5; + private static final int TASKS_MAX = 1; + + private static final String CONNECTOR_NAME = "es-connector"; + private static final String TOPIC = "test"; + private Map props; + + @Before + public void setup() { + startConnect(); + connect.kafka().createTopic(TOPIC); + props = createProps(); + + stubFor(any(anyUrl()).atPriority(10).willReturn(ok())); + } + + @After + public void cleanup() { + stopConnect(); + } + + /** + * Transformer that blocks all incoming requests until {@link #release(int)} is called + * to fairly unblock a given number of requests. + */ + public static class ConcurrencyTransformer extends ResponseTransformer { + + private final Semaphore s = new Semaphore(0, true); + private final AtomicInteger requestCount = new AtomicInteger(); + + @Override + public Response transform(Request request, Response response, FileSource files, Parameters parameters) { + try { + s.acquire(); + } catch (InterruptedException e) { + throw new ConnectException(e); + } finally { + s.release(); + } + requestCount.incrementAndGet(); + return response; + } + + @Override + public String getName() { + return "concurrency"; + } + + public void release(int permits) { + s.release(permits); + } + + /** + * How many requests are currently blocked + */ + public int queueLength() { + return s.getQueueLength(); + } + + /** + * How many requests have been processed + */ + public int requestCount() { + return requestCount.get(); + } + + @Override + public boolean applyGlobally() { + return false; + } + + } + + @Test + public void testRetry() throws Exception { + wireMockRule.stubFor(post(urlPathEqualTo("/_bulk")) + .inScenario("bulkRetry1") + .whenScenarioStateIs(Scenario.STARTED) + .withRequestBody(WireMock.containing("{\"doc_num\":0}")) + .willReturn(aResponse().withStatus(500)) + .willSetStateTo("Failed")); + + wireMockRule.stubFor(post(urlPathEqualTo("/_bulk")) + .inScenario("bulkRetry1") + .whenScenarioStateIs("Failed") + .withRequestBody(WireMock.containing("{\"doc_num\":0}")) + .willSetStateTo("Fixed") + .willReturn(okJson(okBulkResponse()))); + + connect.configureConnector(CONNECTOR_NAME, props); + waitForConnectorToStart(CONNECTOR_NAME, TASKS_MAX); + writeRecords(4); + + await().untilAsserted( + () -> assertThat(wireMockRule.getAllScenarios().getScenarios().get(0).getState()) + .isEqualTo("Fixed")); + + assertThat(connect.connectorStatus(CONNECTOR_NAME).tasks().get(0).state()) + .isEqualTo("RUNNING"); + } + + @Test + public void testConcurrentRequests() throws Exception { + ConcurrencyTransformer concurrencyTransformer = new ConcurrencyTransformer(); + WireMockServer wireMockServer = new WireMockServer(options().dynamicPort() + .extensions(concurrencyTransformer)); + + try { + wireMockServer.start(); + wireMockServer.stubFor(post(urlPathEqualTo("/_bulk")) + .willReturn(okJson(okBulkResponse()) + .withTransformers(concurrencyTransformer.getName()))); + wireMockServer.stubFor(any(anyUrl()).atPriority(10).willReturn(ok())); + + props.put(CONNECTION_URL_CONFIG, wireMockServer.url("/")); + props.put(READ_TIMEOUT_MS_CONFIG, "60000"); + props.put(MAX_RETRIES_CONFIG, "0"); + props.put(LINGER_MS_CONFIG, "60000"); + props.put(BATCH_SIZE_CONFIG, "1"); + props.put(MAX_IN_FLIGHT_REQUESTS_CONFIG, "4"); + + connect.configureConnector(CONNECTOR_NAME, props); + waitForConnectorToStart(CONNECTOR_NAME, TASKS_MAX); + writeRecords(10); + + // TODO MAX_IN_FLIGHT_REQUESTS_CONFIG is misleading (it allows 1 less concurrent request + // than configure), but fixing it would be a breaking change. + // Consider allowing 0 (blocking) and removing "-1" + await().atMost(Duration.ofSeconds(10)).untilAsserted(() -> + assertThat(concurrencyTransformer.queueLength()).isEqualTo(3)); + + concurrencyTransformer.release(10); + + await().atMost(Duration.ofSeconds(10)).untilAsserted(() -> + assertThat(concurrencyTransformer.requestCount()).isEqualTo(10)); + } finally { + wireMockServer.stop(); + } + } + + @Ignore("flaky") + @Test + public void testReadTimeout() throws Exception { + wireMockRule.stubFor(post(urlPathEqualTo("/_bulk")) + .willReturn(ok().withFixedDelay(2_000))); + + connect.configureConnector(CONNECTOR_NAME, props); + waitForConnectorToStart(CONNECTOR_NAME, TASKS_MAX); + writeRecords(NUM_RECORDS); + + // Connector should fail since the request takes longer than request timeout + await().atMost(Duration.ofMinutes(1)).untilAsserted(() -> + assertThat(connect.connectorStatus(CONNECTOR_NAME).tasks().get(0).state()) + .isEqualTo("FAILED")); + + assertThat(connect.connectorStatus(CONNECTOR_NAME).tasks().get(0).trace()) + .contains("Failed to execute bulk request due to 'java.net.SocketTimeoutException: " + + "1,000 milliseconds timeout on connection") + .contains("after 3 attempt(s)"); + + // 1 + 2 retries + verify(3, postRequestedFor(urlPathEqualTo("/_bulk"))); + } + + @Test + public void testTooManyRequests() throws Exception { + wireMockRule.stubFor(post(urlPathEqualTo("/_bulk")) + .willReturn(aResponse() + .withStatus(429) + .withHeader(CONTENT_TYPE, "application/json") + .withBody("{\n" + + " \"error\": {\n" + + " \"type\": \"circuit_breaking_exception\",\n" + + " \"reason\": \"Data too large\",\n" + + " \"bytes_wanted\": 123848638,\n" + + " \"bytes_limit\": 123273216,\n" + + " \"durability\": \"TRANSIENT\"\n" + + " },\n" + + " \"status\": 429\n" + + "}"))); + + connect.configureConnector(CONNECTOR_NAME, props); + waitForConnectorToStart(CONNECTOR_NAME, TASKS_MAX); + writeRecords(NUM_RECORDS); + + // Connector should fail since the request takes longer than request timeout + await().atMost(Duration.ofMinutes(1)).untilAsserted(() -> + assertThat(connect.connectorStatus(CONNECTOR_NAME).tasks().get(0).state()) + .isEqualTo("FAILED")); + + assertThat(connect.connectorStatus(CONNECTOR_NAME).tasks().get(0).trace()) + .contains("Failed to execute bulk request due to 'ElasticsearchStatusException" + + "[Elasticsearch exception [type=circuit_breaking_exception, " + + "reason=Data too large]]' after 3 attempt(s)"); + + // 1 + 2 retries + verify(3, postRequestedFor(urlPathEqualTo("/_bulk"))); + } + + @Test + public void testServiceUnavailable() throws Exception { + wireMockRule.stubFor(post(urlPathEqualTo("/_bulk")) + .willReturn(aResponse() + .withStatus(503))); + + connect.configureConnector(CONNECTOR_NAME, props); + waitForConnectorToStart(CONNECTOR_NAME, TASKS_MAX); + writeRecords(NUM_RECORDS); + + // Connector should fail since the request takes longer than request timeout + await().atMost(Duration.ofMinutes(1)).untilAsserted(() -> + assertThat(connect.connectorStatus(CONNECTOR_NAME).tasks().get(0).state()) + .isEqualTo("FAILED")); + + assertThat(connect.connectorStatus(CONNECTOR_NAME).tasks().get(0).trace()) + .contains("[HTTP/1.1 503 Service Unavailable]") + .contains("after 3 attempt(s)"); + + // 1 + 2 retries + verify(3, postRequestedFor(urlPathEqualTo("/_bulk"))); + } + + protected Map createProps() { + Map props = new HashMap<>(); + + // generic configs + props.put(CONNECTOR_CLASS_CONFIG, ElasticsearchSinkConnector.class.getName()); + props.put(TOPICS_CONFIG, TOPIC); + props.put(TASKS_MAX_CONFIG, Integer.toString(TASKS_MAX)); + props.put(KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); + props.put(VALUE_CONVERTER_CLASS_CONFIG, JsonConverter.class.getName()); + props.put("value.converter." + SCHEMAS_ENABLE_CONFIG, "false"); + + // connectors specific + props.put(CONNECTION_URL_CONFIG, wireMockRule.url("/")); + props.put(IGNORE_KEY_CONFIG, "true"); + props.put(IGNORE_SCHEMA_CONFIG, "true"); + + props.put(READ_TIMEOUT_MS_CONFIG, "1000"); + props.put(MAX_RETRIES_CONFIG, "2"); + props.put(RETRY_BACKOFF_MS_CONFIG, "10"); + props.put(LINGER_MS_CONFIG, "60000"); + props.put(BATCH_SIZE_CONFIG, "4"); + props.put(MAX_IN_FLIGHT_REQUESTS_CONFIG, "1"); + + return props; + } + + protected void writeRecords(int numRecords) { + for (int i = 0; i < numRecords; i++) { + connect.kafka().produce(TOPIC, String.valueOf(i), String.format("{\"doc_num\":%d}", i)); + } + } + + private String okBulkResponse() throws JsonProcessingException { + ObjectNode response = MAPPER.createObjectNode(); + response + .put("errors", false) + .putArray("items") + .addObject() + .putObject("index") + .put("_index", "test") + .put("_type", "_doc") + .put("_id", "1") + .put("_version", "1") + .put("result", "created") + .put("status", 201) + .put("_seq_no", 0); + return MAPPER.writerWithDefaultPrettyPrinter().writeValueAsString(response); + } + +} diff --git a/src/test/java/io/confluent/connect/elasticsearch/integration/ElasticsearchConnectorSecureIT.java b/src/test/java/io/confluent/connect/elasticsearch/integration/ElasticsearchConnectorSslIT.java similarity index 61% rename from src/test/java/io/confluent/connect/elasticsearch/integration/ElasticsearchConnectorSecureIT.java rename to src/test/java/io/confluent/connect/elasticsearch/integration/ElasticsearchConnectorSslIT.java index e7f437e36..4e09894ed 100644 --- a/src/test/java/io/confluent/connect/elasticsearch/integration/ElasticsearchConnectorSecureIT.java +++ b/src/test/java/io/confluent/connect/elasticsearch/integration/ElasticsearchConnectorSslIT.java @@ -16,28 +16,38 @@ package io.confluent.connect.elasticsearch.integration; import io.confluent.common.utils.IntegrationTest; -import io.confluent.connect.elasticsearch.ElasticsearchClient; -import io.confluent.connect.elasticsearch.SecurityProtocol; -import io.confluent.connect.elasticsearch.jest.JestElasticsearchClient; +import io.confluent.connect.elasticsearch.helper.ElasticsearchContainer; + import org.apache.kafka.common.config.SslConfigs; +import org.elasticsearch.client.security.user.User; +import org.elasticsearch.client.security.user.privileges.Role; import org.junit.BeforeClass; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.CONNECTION_PASSWORD_CONFIG; import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.CONNECTION_URL_CONFIG; -import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.SECURITY_PROTOCOL_CONFIG; +import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.CONNECTION_USERNAME_CONFIG; import static io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig.SSL_CONFIG_PREFIX; + @Category(IntegrationTest.class) -public class ElasticsearchConnectorSecureIT extends ElasticsearchConnectorBaseIT { +public class ElasticsearchConnectorSslIT extends ElasticsearchConnectorBaseIT { - private static final Logger log = LoggerFactory.getLogger(ElasticsearchConnectorSecureIT.class); + private static final Logger log = LoggerFactory.getLogger(ElasticsearchConnectorSslIT.class); @BeforeClass public static void setupBeforeAll() { - container = ElasticsearchContainer.fromSystemProperties().withSslEnabled(true); + Map users = Collections + .singletonMap(getMinimalPrivilegesUser(), getMinimalPrivilegesPassword()); + List roles = Collections.singletonList(getMinimalPrivilegesRole()); + container = ElasticsearchContainer.fromSystemProperties().withSslEnabled(true).withBasicAuth(users, roles); container.start(); } @@ -47,20 +57,27 @@ public static void setupBeforeAll() { */ @Test public void testSecureConnectionVerifiedHostname() throws Throwable { - // Use IP address here because that's what the certificates allow - String address = container.getConnectionUrl(); - address = address.replace(container.getContainerIpAddress(), container.hostMachineIpAddress()); + // Use container IP address here because that's what the certificates allow + String address = container.getConnectionUrl(false); log.info("Creating connector for {}.", address); props.put(CONNECTION_URL_CONFIG, address); - addSslProps(); + container.addSslProps(props); - client = new JestElasticsearchClient(props); + helperClient = container.getHelperClient(props); // Start connector runSimpleTest(props); } + @Override + protected Map createProps() { + props = super.createProps(); + props.put(CONNECTION_USERNAME_CONFIG, ELASTIC_MINIMAL_PRIVILEGES_NAME); + props.put(CONNECTION_PASSWORD_CONFIG, ELASTIC_MINIMAL_PRIVILEGES_PASSWORD); + return props; + } + @Test public void testSecureConnectionHostnameVerificationDisabled() throws Throwable { // Use 'localhost' here that is not in self-signed cert @@ -69,29 +86,14 @@ public void testSecureConnectionHostnameVerificationDisabled() throws Throwable log.info("Creating connector for {}", address); props.put(CONNECTION_URL_CONFIG, address); - addSslProps(); + container.addSslProps(props); // disable hostname verification props.put(SSL_CONFIG_PREFIX + SslConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG, ""); - client = new JestElasticsearchClient(props); + helperClient = container.getHelperClient(props); // Start connector runSimpleTest(props); } - - @Override - protected ElasticsearchClient createClient() { - // will be created in the test with the proper SSL configs - return null; - } - - private void addSslProps() { - props.put(SECURITY_PROTOCOL_CONFIG, SecurityProtocol.SSL.name()); - props.put(SSL_CONFIG_PREFIX + SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG, container.getKeystorePath()); - props.put(SSL_CONFIG_PREFIX + SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, container.getKeystorePassword()); - props.put(SSL_CONFIG_PREFIX + SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, container.getTruststorePath()); - props.put(SSL_CONFIG_PREFIX + SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, container.getTruststorePassword()); - props.put(SSL_CONFIG_PREFIX + SslConfigs.SSL_KEY_PASSWORD_CONFIG, container.getKeyPassword()); - } } diff --git a/src/test/java/io/confluent/connect/elasticsearch/jest/JestElasticsearchClientTest.java b/src/test/java/io/confluent/connect/elasticsearch/jest/JestElasticsearchClientTest.java deleted file mode 100644 index 66f22aade..000000000 --- a/src/test/java/io/confluent/connect/elasticsearch/jest/JestElasticsearchClientTest.java +++ /dev/null @@ -1,477 +0,0 @@ -/* - * 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.connect.elasticsearch.jest; - -import com.fasterxml.jackson.databind.node.JsonNodeFactory; -import com.fasterxml.jackson.databind.node.ObjectNode; -import com.google.gson.Gson; -import com.google.gson.JsonArray; -import com.google.gson.JsonObject; -import io.confluent.connect.elasticsearch.ElasticsearchClient; -import io.confluent.connect.elasticsearch.ElasticsearchSinkConnectorConfig; -import io.confluent.connect.elasticsearch.IndexableRecord; -import io.confluent.connect.elasticsearch.Key; -import io.confluent.connect.elasticsearch.Mapping; -import io.confluent.connect.elasticsearch.bulk.BulkRequest; -import io.searchbox.action.BulkableAction; -import io.searchbox.client.JestClient; -import io.searchbox.client.JestClientFactory; -import io.searchbox.client.JestResult; -import io.searchbox.client.config.ElasticsearchVersion; -import io.searchbox.client.config.HttpClientConfig; -import io.searchbox.cluster.NodesInfo; -import io.searchbox.core.BulkResult; -import io.searchbox.core.Delete; -import io.searchbox.core.DocumentResult; -import io.searchbox.core.Index; -import io.searchbox.core.Search; -import io.searchbox.core.SearchResult; -import io.searchbox.core.Update; -import io.searchbox.indices.CreateIndex; -import io.searchbox.indices.IndicesExists; -import io.searchbox.indices.mapping.GetMapping; -import io.searchbox.indices.mapping.PutMapping; -import java.io.IOException; -import org.apache.http.HttpHost; -import org.apache.http.auth.AuthScope; -import org.apache.http.auth.Credentials; -import org.apache.http.client.CredentialsProvider; -import org.apache.http.conn.routing.HttpRoutePlanner; -import org.apache.http.impl.conn.DefaultProxyRoutePlanner; -import org.apache.kafka.connect.data.Schema; -import org.apache.kafka.connect.errors.ConnectException; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.mockito.ArgumentCaptor; -import org.mockito.ArgumentMatcher; -import org.mockito.InOrder; - -import java.lang.reflect.Field; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; - -import static io.confluent.connect.elasticsearch.jest.JestElasticsearchClient.RESOURCE_ALREADY_EXISTS_EXCEPTION; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.is; -import static org.hamcrest.Matchers.equalTo; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertSame; -import static org.junit.Assert.assertTrue; -import static org.mockito.ArgumentMatchers.argThat; -import static org.mockito.Mockito.inOrder; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.verifyNoMoreInteractions; -import static org.mockito.Mockito.when; - -public class JestElasticsearchClientTest { - - private static final String INDEX = "index"; - private static final String KEY = "key"; - private static final String TYPE = "type"; - private static final String QUERY = "query"; - - private JestClient jestClient; - private JestClientFactory jestClientFactory; - private NodesInfo info; - - @Before - public void setUp() throws Exception { - jestClient = mock(JestClient.class); - jestClientFactory = mock(JestClientFactory.class); - when(jestClientFactory.getObject()).thenReturn(jestClient); - info = new NodesInfo.Builder().addCleanApiParameter("version").build(); - JsonObject nodeRoot = new JsonObject(); - nodeRoot.addProperty("version", "1.0"); - JsonObject nodesRoot = new JsonObject(); - nodesRoot.add("localhost", nodeRoot); - JsonObject nodesInfo = new JsonObject(); - nodesInfo.add("nodes", nodesRoot); - JestResult result = new JestResult(new Gson()); - result.setJsonObject(nodesInfo); - when(jestClient.execute(info)).thenReturn(result); - } - - @Test - public void connectsSecurely() { - Map props = new HashMap<>(); - props.put(ElasticsearchSinkConnectorConfig.CONNECTION_URL_CONFIG, "http://localhost:9200"); - props.put(ElasticsearchSinkConnectorConfig.CONNECTION_USERNAME_CONFIG, "elastic"); - props.put(ElasticsearchSinkConnectorConfig.CONNECTION_PASSWORD_CONFIG, "elasticpw"); - props.put(ElasticsearchSinkConnectorConfig.TYPE_NAME_CONFIG, "kafka-connect"); - JestElasticsearchClient client = new JestElasticsearchClient(props, jestClientFactory); - - ArgumentCaptor captor = ArgumentCaptor.forClass(HttpClientConfig.class); - verify(jestClientFactory).setHttpClientConfig(captor.capture()); - HttpClientConfig httpClientConfig = captor.getValue(); - CredentialsProvider credentialsProvider = httpClientConfig.getCredentialsProvider(); - Credentials credentials = credentialsProvider.getCredentials(AuthScope.ANY); - Set preemptiveAuthTargetHosts = httpClientConfig.getPreemptiveAuthTargetHosts(); - assertEquals("elastic", credentials.getUserPrincipal().getName()); - assertEquals("elasticpw", credentials.getPassword()); - assertEquals(HttpHost.create("http://localhost:9200"), preemptiveAuthTargetHosts.iterator().next()); - } - - @Test - public void connectsWithProxy() throws NoSuchFieldException, IllegalAccessException { - Map props = new HashMap<>(); - props.put(ElasticsearchSinkConnectorConfig.CONNECTION_URL_CONFIG, "http://localhost:9200"); - props.put(ElasticsearchSinkConnectorConfig.PROXY_HOST_CONFIG, "myproxy"); - props.put(ElasticsearchSinkConnectorConfig.PROXY_PORT_CONFIG, "443"); - props.put(ElasticsearchSinkConnectorConfig.PROXY_USERNAME_CONFIG, "username"); - props.put(ElasticsearchSinkConnectorConfig.PROXY_PASSWORD_CONFIG, "password"); - - props.put(ElasticsearchSinkConnectorConfig.TYPE_NAME_CONFIG, "kafka-connect"); - JestElasticsearchClient client = new JestElasticsearchClient(props, jestClientFactory); - - ArgumentCaptor captor = ArgumentCaptor.forClass(HttpClientConfig.class); - verify(jestClientFactory).setHttpClientConfig(captor.capture()); - HttpClientConfig httpClientConfig = captor.getValue(); - HttpRoutePlanner routePlanner = httpClientConfig.getHttpRoutePlanner(); - - assertTrue(routePlanner instanceof DefaultProxyRoutePlanner); - DefaultProxyRoutePlanner proxyRoutePlanner = (DefaultProxyRoutePlanner) routePlanner; - - Field f = proxyRoutePlanner.getClass().getDeclaredField("proxy"); - f.setAccessible(true); - HttpHost httpProxy = (HttpHost) f.get(proxyRoutePlanner); - - assertEquals("http", httpProxy.getSchemeName()); - assertEquals("myproxy", httpProxy.getHostName()); - assertEquals(443, httpProxy.getPort()); - - Credentials credentials = httpClientConfig - .getCredentialsProvider() - .getCredentials(new AuthScope(httpProxy)); - - assertEquals("password", credentials.getPassword()); - } - - @Test - public void compressedConnectsSecurely() { - Map props = new HashMap<>(); - props.put(ElasticsearchSinkConnectorConfig.CONNECTION_URL_CONFIG, "http://localhost:9200"); - props.put(ElasticsearchSinkConnectorConfig.CONNECTION_USERNAME_CONFIG, "elastic"); - props.put(ElasticsearchSinkConnectorConfig.CONNECTION_PASSWORD_CONFIG, "elasticpw"); - props.put(ElasticsearchSinkConnectorConfig.TYPE_NAME_CONFIG, "kafka-connect"); - props.put(ElasticsearchSinkConnectorConfig.CONNECTION_COMPRESSION_CONFIG, "true"); - JestElasticsearchClient client = new JestElasticsearchClient(props, jestClientFactory); - - ArgumentCaptor captor = ArgumentCaptor.forClass(HttpClientConfig.class); - verify(jestClientFactory).setHttpClientConfig(captor.capture()); - HttpClientConfig httpClientConfig = captor.getValue(); - assertTrue(httpClientConfig.isRequestCompressionEnabled()); - } - - @Test - public void connectsSecurelyWithEmptyUsernameAndPassword() { - Map props = new HashMap<>(); - props.put(ElasticsearchSinkConnectorConfig.CONNECTION_URL_CONFIG, "http://localhost:9200"); - props.put(ElasticsearchSinkConnectorConfig.CONNECTION_USERNAME_CONFIG, ""); - props.put(ElasticsearchSinkConnectorConfig.CONNECTION_PASSWORD_CONFIG, ""); - props.put(ElasticsearchSinkConnectorConfig.TYPE_NAME_CONFIG, "kafka-connect"); - JestElasticsearchClient client = new JestElasticsearchClient(props, jestClientFactory); - - ArgumentCaptor captor = ArgumentCaptor.forClass(HttpClientConfig.class); - verify(jestClientFactory).setHttpClientConfig(captor.capture()); - HttpClientConfig httpClientConfig = captor.getValue(); - CredentialsProvider credentialsProvider = httpClientConfig.getCredentialsProvider(); - Credentials credentials = credentialsProvider.getCredentials(AuthScope.ANY); - Set preemptiveAuthTargetHosts = httpClientConfig.getPreemptiveAuthTargetHosts(); - assertEquals("", credentials.getUserPrincipal().getName()); - assertEquals("", credentials.getPassword()); - assertEquals(HttpHost.create("http://localhost:9200"), preemptiveAuthTargetHosts.iterator().next()); - } - - @Test - public void getsVersion() { - JestElasticsearchClient client = new JestElasticsearchClient(jestClient); - assertThat(client.getVersion(), is(equalTo(ElasticsearchClient.Version.ES_V1))); - } - - @Test - public void attemptToCreateExistingIndex() throws Exception { - JestElasticsearchClient client = new JestElasticsearchClient(jestClient); - JestResult success = new JestResult(new Gson()); - success.setSucceeded(true); - IndicesExists indicesExists = new IndicesExists.Builder(INDEX).build(); - when(jestClient.execute(indicesExists)).thenReturn(success); - when(jestClient.execute(argThat(isCreateIndexForTestIndex()))).thenReturn(success); - - client.createIndices(Collections.singleton(INDEX)); - InOrder inOrder = inOrder(jestClient); - inOrder.verify(jestClient).execute(info); - inOrder.verify(jestClient).execute(indicesExists); - - verifyNoMoreInteractions(jestClient); - } - - @Test - public void createsIndices() throws Exception { - JestElasticsearchClient client = new JestElasticsearchClient(jestClient); - JestResult failure = new JestResult(new Gson()); - failure.setSucceeded(false); - JestResult success = new JestResult(new Gson()); - success.setSucceeded(true); - IndicesExists indicesExists = new IndicesExists.Builder(INDEX).build(); - when(jestClient.execute(indicesExists)).thenReturn(failure); - when(jestClient.execute(argThat(isCreateIndexForTestIndex()))).thenReturn(success); - - Set indices = new HashSet<>(); - indices.add(INDEX); - client.createIndices(indices); - InOrder inOrder = inOrder(jestClient); - inOrder.verify(jestClient).execute(info); - inOrder.verify(jestClient).execute(indicesExists); - inOrder.verify(jestClient).execute(argThat(isCreateIndexForTestIndex())); - } - - private ArgumentMatcher isCreateIndexForTestIndex() { - return new ArgumentMatcher() { - @Override - public boolean matches(CreateIndex createIndex) { - // check the URI as the equals method on CreateIndex doesn't work - return createIndex.getURI(ElasticsearchVersion.V2).equals(INDEX); - } - }; - } - - @Test(expected = ConnectException.class) - public void createIndicesAndFails() throws Exception { - JestElasticsearchClient client = new JestElasticsearchClient(jestClient); - JestResult failure = new JestResult(new Gson()); - failure.setSucceeded(false); - failure.setErrorMessage("unrelated error"); - IndicesExists indicesExists = new IndicesExists.Builder(INDEX).build(); - when(jestClient.execute(indicesExists)).thenReturn(failure); - when(jestClient.execute(argThat(isCreateIndexForTestIndex()))).thenReturn(failure); - - Set indices = new HashSet<>(); - indices.add(INDEX); - client.createIndices(indices); - } - - @Test - public void createIndexAlreadyExists() throws Exception { - JestElasticsearchClient client = new JestElasticsearchClient(jestClient); - JestResult failure = new JestResult(new Gson()); - failure.setSucceeded(false); - failure.setErrorMessage(RESOURCE_ALREADY_EXISTS_EXCEPTION); - JestResult success = new JestResult(new Gson()); - success.setSucceeded(true); - IndicesExists indicesExists = new IndicesExists.Builder(INDEX).build(); - when(jestClient.execute(indicesExists)).thenReturn(failure); - when(jestClient.execute(argThat(isCreateIndexForTestIndex()))).thenReturn(success); - - Set indices = new HashSet<>(); - indices.add(INDEX); - client.createIndices(indices); - - InOrder inOrder = inOrder(jestClient); - inOrder.verify(jestClient).execute(info); - inOrder.verify(jestClient).execute(indicesExists); - inOrder.verify(jestClient).execute(argThat(isCreateIndexForTestIndex())); - } - - @Test - public void createsMapping() throws Exception { - JestElasticsearchClient client = new JestElasticsearchClient(jestClient); - JestResult success = new JestResult(new Gson()); - success.setSucceeded(true); - ObjectNode obj = JsonNodeFactory.instance.objectNode(); - obj.set(TYPE, Mapping.inferMapping(client, Schema.STRING_SCHEMA)); - PutMapping putMapping = new PutMapping.Builder(INDEX, TYPE,obj.toString()).build(); - when(jestClient.execute(putMapping)).thenReturn(success); - - client.createMapping(INDEX, TYPE, Schema.STRING_SCHEMA); - verify(jestClient).execute(putMapping); - } - - @Test(expected = ConnectException.class) - public void createsMappingAndFails() throws Exception { - JestElasticsearchClient client = new JestElasticsearchClient(jestClient); - JestResult failure = new JestResult(new Gson()); - failure.setSucceeded(false); - ObjectNode obj = JsonNodeFactory.instance.objectNode(); - obj.set(TYPE, Mapping.inferMapping(client, Schema.STRING_SCHEMA)); - PutMapping putMapping = new PutMapping.Builder(INDEX, TYPE, obj.toString()).build(); - when(jestClient.execute(putMapping)).thenReturn(failure); - - client.createMapping(INDEX, TYPE, Schema.STRING_SCHEMA); - } - - @Test - public void getsMapping() throws Exception { - JestElasticsearchClient client = new JestElasticsearchClient(jestClient); - JsonObject mapping = new JsonObject(); - JsonObject mappings = new JsonObject(); - mappings.add(TYPE, mapping); - JsonObject indexRoot = new JsonObject(); - indexRoot.add("mappings", mappings); - JsonObject root = new JsonObject(); - root.add(INDEX, indexRoot); - JestResult result = new JestResult(new Gson()); - result.setJsonObject(root); - GetMapping getMapping = new GetMapping.Builder().addIndex(INDEX).addType(TYPE).build(); - when(jestClient.execute(getMapping)).thenReturn(result); - - assertThat(client.getMapping(INDEX, TYPE), is(equalTo(mapping))); - } - - @Test - public void executesBulk() throws Exception { - JestElasticsearchClient client = new JestElasticsearchClient(jestClient); - IndexableRecord record = new IndexableRecord(new Key(INDEX, TYPE, KEY), "payload", 0L); - List records = new ArrayList<>(); - records.add(record); - BulkRequest request = client.createBulkRequest(records); - BulkResult success = new BulkResult(new Gson()); - success.setSucceeded(true); - when(jestClient.execute(((JestBulkRequest) request).getBulk())).thenReturn(success); - - assertThat(client.executeBulk(request).isSucceeded(), is(equalTo(true))); - } - - @Test - public void executesBulkAndFails() throws Exception { - JestElasticsearchClient client = new JestElasticsearchClient(jestClient); - IndexableRecord record = new IndexableRecord(new Key(INDEX, TYPE, KEY), null, 0L); - List records = new ArrayList<>(); - records.add(record); - BulkRequest request = client.createBulkRequest(records); - BulkResult failure = new BulkResult(new Gson()); - failure.setSucceeded(false); - when(jestClient.execute(((JestBulkRequest) request).getBulk())).thenReturn(failure); - - assertThat(client.executeBulk(request).isSucceeded(), is(equalTo(false))); - } - - @Test - public void executesBulkAndFailsWithParseError() throws Exception { - JestElasticsearchClient client = new JestElasticsearchClient(jestClient); - IndexableRecord record = new IndexableRecord(new Key(INDEX, TYPE, KEY), "payload", 0L); - List records = new ArrayList<>(); - records.add(record); - BulkRequest request = client.createBulkRequest(records); - BulkResult failure = createBulkResultFailure(JestElasticsearchClient.MAPPER_PARSE_EXCEPTION); - when(jestClient.execute(((JestBulkRequest) request).getBulk())).thenReturn(failure); - - assertThat(client.executeBulk(request).isSucceeded(), is(equalTo(false))); - } - - @Test - public void executesBulkAndFailsWithSomeOtherError() throws Exception { - JestElasticsearchClient client = new JestElasticsearchClient(jestClient); - IndexableRecord record = new IndexableRecord(new Key(INDEX, TYPE, KEY), "payload", 0L); - List records = new ArrayList<>(); - records.add(record); - BulkRequest request = client.createBulkRequest(records); - BulkResult failure = createBulkResultFailure("some_random_exception"); - when(jestClient.execute(((JestBulkRequest) request).getBulk())).thenReturn(failure); - - assertThat(client.executeBulk(request).isSucceeded(), is(equalTo(false))); - } - - @Test - public void executesBulkAndSucceedsBecauseOnlyVersionConflicts() throws Exception { - JestElasticsearchClient client = new JestElasticsearchClient(jestClient); - IndexableRecord record = new IndexableRecord(new Key(INDEX, TYPE, KEY), "payload", 0L); - List records = new ArrayList<>(); - records.add(record); - BulkRequest request = client.createBulkRequest(records); - BulkResult failure = createBulkResultFailure(JestElasticsearchClient.VERSION_CONFLICT_ENGINE_EXCEPTION); - when(jestClient.execute(((JestBulkRequest) request).getBulk())).thenReturn(failure); - - assertThat(client.executeBulk(request).isSucceeded(), is(equalTo(true))); - } - - @Test - public void searches() throws Exception { - JestElasticsearchClient client = new JestElasticsearchClient(jestClient); - Search search = new Search.Builder(QUERY).addIndex(INDEX).addType(TYPE).build(); - JsonObject queryResult = new JsonObject(); - SearchResult result = new SearchResult(new Gson()); - result.setJsonObject(queryResult); - when(jestClient.execute(search)).thenReturn(result); - - assertThat(client.search(QUERY, INDEX, TYPE), is(equalTo(queryResult))); - } - - @Test - public void closes() throws IOException { - JestElasticsearchClient client = new JestElasticsearchClient(jestClient); - client.close(); - - verify(jestClient).close(); - } - - @Test - public void toBulkableAction(){ - JestElasticsearchClient client = new JestElasticsearchClient(jestClient); - IndexableRecord del = new IndexableRecord(new Key("idx", "tp", "xxx"), null, 1L); - BulkableAction ba = client.toBulkableAction(del); - assertNotNull(ba); - assertSame(Delete.class, ba.getClass()); - assertEquals(del.key.index, ba.getIndex()); - assertEquals(del.key.id, ba.getId()); - assertEquals(del.key.type, ba.getType()); - IndexableRecord idx = new IndexableRecord(new Key("idx", "tp", "xxx"), "yyy", 1L); - ba = client.toBulkableAction(idx); - assertNotNull(ba); - assertSame(Index.class, ba.getClass()); - assertEquals(idx.key.index, ba.getIndex()); - assertEquals(idx.key.id, ba.getId()); - assertEquals(idx.key.type, ba.getType()); - assertEquals(idx.payload, ba.getData(null)); - // upsert - client.setWriteMethod(JestElasticsearchClient.WriteMethod.UPSERT); - ba = client.toBulkableAction(idx); - assertNotNull(ba); - assertSame(Update.class, ba.getClass()); - assertEquals(idx.key.index, ba.getIndex()); - assertEquals(idx.key.id, ba.getId()); - assertEquals(idx.key.type, ba.getType()); - assertEquals("{\"doc\":" + idx.payload + ", \"doc_as_upsert\":true}", ba.getData(null)); - } - - private BulkResult createBulkResultFailure(String exception) { - BulkResult failure = new BulkResult(new Gson()); - failure.setSucceeded(false); - JsonObject error = new JsonObject(); - error.addProperty("type", exception); - JsonObject item = new JsonObject(); - item.addProperty("_index", INDEX); - item.addProperty("_type", TYPE); - item.addProperty("status", 0); - item.add("error", error); - JsonObject index = new JsonObject(); - index.add("index", item); - JsonArray items = new JsonArray(); - items.add(index); - JsonObject root = new JsonObject(); - root.add("items", items); - failure.setJsonObject(root); - return failure; - } -} diff --git a/src/test/resources/basic/elasticsearch.yml b/src/test/resources/basic/elasticsearch.yml new file mode 100755 index 000000000..8f30245aa --- /dev/null +++ b/src/test/resources/basic/elasticsearch.yml @@ -0,0 +1,5 @@ +cluster.name: "docker-cluster" +network.host: 0.0.0.0 + +xpack.license.self_generated.type: trial +xpack.security.enabled: true diff --git a/src/test/resources/basic/instances.yml b/src/test/resources/basic/instances.yml new file mode 100644 index 000000000..8f5cefcc3 --- /dev/null +++ b/src/test/resources/basic/instances.yml @@ -0,0 +1,18 @@ +instances: + - name: elasticsearch + dns: + - elasticsearch + ip: + - ipAddress + - name: kibana + dns: + - kibana + - localhost + ip: + - ipAddress + - name: logstash + dns: + - logstash + - localhost + ip: + - ipAddress diff --git a/src/test/resources/both/elasticsearch.yml b/src/test/resources/both/elasticsearch.yml new file mode 100755 index 000000000..02d62d3d5 --- /dev/null +++ b/src/test/resources/both/elasticsearch.yml @@ -0,0 +1,34 @@ +## Used by Docker images in our integration test +http.host: 0.0.0.0 +network.host: 0.0.0.0 +transport.host: 0.0.0.0 + +node.store.allow_mmap: false +cluster.routing.allocation.disk.threshold_enabled: false + +xpack.license.self_generated.type: trial +xpack.security.enabled: true +xpack.security.http.ssl.enabled: true +xpack.security.http.ssl.client_authentication: optional +xpack.security.http.ssl.verification_mode: certificate +xpack.security.http.ssl.key: ssl/elasticsearch.key +xpack.security.http.ssl.certificate: ssl/elasticsearch.crt +xpack.security.http.ssl.certificate_authorities: [ "ssl/ca/ca.crt" ] + +xpack.security.transport.ssl.enabled: true +xpack.security.transport.ssl.key: ssl/elasticsearch.key +xpack.security.transport.ssl.certificate: ssl/elasticsearch.crt +xpack.security.transport.ssl.certificate_authorities: [ "ssl/ca/ca.crt" ] + +# enable anonymous connections since setting passwords requires running a command +xpack.security.authc: + anonymous: + username: connect_user + roles: superuser + authz_exception: true + +# Kerberos realm +xpack.security.authc.realms.kerberos.kerb1: + order: 3 + keytab.path: es.keytab + remove_realm_name: false \ No newline at end of file diff --git a/src/test/resources/both/instances.yml b/src/test/resources/both/instances.yml new file mode 100644 index 000000000..8f5cefcc3 --- /dev/null +++ b/src/test/resources/both/instances.yml @@ -0,0 +1,18 @@ +instances: + - name: elasticsearch + dns: + - elasticsearch + ip: + - ipAddress + - name: kibana + dns: + - kibana + - localhost + ip: + - ipAddress + - name: logstash + dns: + - logstash + - localhost + ip: + - ipAddress diff --git a/src/test/resources/both/start-elasticsearch.sh b/src/test/resources/both/start-elasticsearch.sh new file mode 100755 index 000000000..2d118fcb0 --- /dev/null +++ b/src/test/resources/both/start-elasticsearch.sh @@ -0,0 +1,88 @@ +#!/bin/bash + +# Generate the certificates used in the HTTPS tests. +# Copy these into the docker image to make available to client (Java class in this repo) and +# elastic server (docker container started in test setup). Volume mounting is unsolved issue. + +ES_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}"/ )" >/dev/null 2>&1 && pwd )" +cd $ES_DIR/../.. +ES_DIR=$(pwd) +export PATH=/usr/share/elasticsearch/jdk/bin/:$PATH + +if [[ -z "${IP_ADDRESS}" ]]; then + IP_ADDRESS=$(hostname -I) +fi + +echo +echo "Replacing the ip address in the ${ES_DIR}/config/ssl/instances.yml file with ${IP_ADDRESS}" +sed -i "s/ipAddress/${IP_ADDRESS}/g" ${ES_DIR}/config/ssl/instances.yml + + +echo "Setting up Elasticsearch and generating certificates in ${ES_DIR}" + +if [[ -n "$ELASTIC_PASSWORD" ]]; then + + echo "=== CREATE Keystore ===" + echo "Elastic password is: $ELASTIC_PASSWORD" + if [ -f ${ES_DIR}/config/elasticsearch.keystore ]; then + echo "Removing old ${ES_DIR}/config/elasticsearch.keystore" + rm ${ES_DIR}/config/elasticsearch.keystore + fi + [[ -f ${ES_DIR}/config/elasticsearch.keystore ]] || (${ES_DIR}/bin/elasticsearch-keystore create) + echo "Setting bootstrap.password..." + (echo "$ELASTIC_PASSWORD" | ${ES_DIR}/bin/elasticsearch-keystore add -x 'bootstrap.password') + + # Create SSL Certs + echo "=== CREATE SSL CERTS ===" + + # check if old cluster-ca.zip exists, if it does remove and create a new one. + if [ -f ${ES_DIR}/config/ssl/cluster-ca.zip ]; then + echo "Removing old ca zip..." + rm ${ES_DIR}/config/ssl/cluster-ca.zip + fi + echo "Creating cluster-ca.zip... (warnings are benign)" + ${ES_DIR}/bin/elasticsearch-certutil ca --pem --silent --out ${ES_DIR}/config/ssl/cluster-ca.zip + + # check if ca directory exists, if does, remove then unzip new files + if [ -d ${ES_DIR}/config/ssl/ca ]; then + echo "CA directory exists, removing..." + rm -rf ${ES_DIR}/config/ssl/ca + fi + + echo "Unzip ca files..." + unzip ${ES_DIR}/config/ssl/cluster-ca.zip -d ${ES_DIR}/config/ssl + rm -f ${ES_DIR}/config/ssl/cluster-ca.zip + + # check if certs zip exist. If it does remove and create a new one. + if [ -f ${ES_DIR}/config/ssl/cluster.zip ]; then + echo "Remove old cluster.zip zip..." + rm ${ES_DIR}/config/ssl/cluster.zip + fi + echo "Create cluster certs zipfile... (warnings are benign)" + ${ES_DIR}/bin/elasticsearch-certutil cert --silent --pem --in ${ES_DIR}/config/ssl/instances.yml --out ${ES_DIR}/config/ssl/cluster.zip --ca-cert ${ES_DIR}/config/ssl/ca/ca.crt --ca-key ${ES_DIR}/config/ssl/ca/ca.key + + if [ -d ${ES_DIR}/config/ssl/docker-cluster ]; then + rm -rf ${ES_DIR}/config/ssl/cluster + fi + echo "Unzipping cluster certs zipfile..." + unzip ${ES_DIR}/config/ssl/cluster.zip -d ${ES_DIR}/config/ssl/cluster + rm -f ${ES_DIR}/config/ssl/cluster.zip + + echo "Move elasticsearch certs to SSL config dir..." + mv ${ES_DIR}/config/ssl/cluster/elasticsearch/* ${ES_DIR}/config/ssl/ + + echo "Generating truststore at ${ES_DIR}/config/ssl/truststore.jks" + keytool -keystore ${ES_DIR}/config/ssl/truststore.jks -import -file ${ES_DIR}/config/ssl/ca/ca.crt -alias cacert -storepass $STORE_PASSWORD -noprompt + + echo "Generating keystore for client at ${ES_DIR}/config/ssl/keystore.jks" + # Generate a new PKCS12 keystore using our CA + openssl pkcs12 -export -in ${ES_DIR}/config/ssl/ca/ca.crt -inkey ${ES_DIR}/config/ssl/ca/ca.key -out ${ES_DIR}/config/ssl/client.p12 -name "clientkey" -passin pass:$STORE_PASSWORD -passout pass:$STORE_PASSWORD + + # Convert the PKCS12 keystore to JKS keytstore + keytool -importkeystore -destkeystore ${ES_DIR}/config/ssl/keystore.jks -deststorepass $STORE_PASSWORD -srckeystore ${ES_DIR}/config/ssl/client.p12 -srcstoretype PKCS12 -srcstorepass $STORE_PASSWORD -noprompt + rm -f ${ES_DIR}/config/ssl/client.p12 +fi + +echo +echo "Starting Elasticsearch with SSL and Kerberos enabled ..." +/usr/local/bin/docker-entrypoint.sh \ No newline at end of file diff --git a/src/test/resources/kerberos/elasticsearch.yml b/src/test/resources/kerberos/elasticsearch.yml new file mode 100755 index 000000000..f644e97b5 --- /dev/null +++ b/src/test/resources/kerberos/elasticsearch.yml @@ -0,0 +1,22 @@ +## Used by Docker images in our integration test +http.host: 0.0.0.0 +network.host: 0.0.0.0 +transport.host: 0.0.0.0 + +node.store.allow_mmap: false +cluster.routing.allocation.disk.threshold_enabled: false + +xpack.license.self_generated.type: trial + +# Kerberos realm +xpack.security.authc.realms.kerberos.kerb1: + order: 3 + keytab.path: es.keytab + remove_realm_name: false + +# enable anonymous connections since setting passwords requires running a command +xpack.security.authc: + anonymous: + username: connect_user + roles: superuser + authz_exception: true \ No newline at end of file diff --git a/src/test/resources/kerberos/instances.yml b/src/test/resources/kerberos/instances.yml new file mode 100644 index 000000000..8f5cefcc3 --- /dev/null +++ b/src/test/resources/kerberos/instances.yml @@ -0,0 +1,18 @@ +instances: + - name: elasticsearch + dns: + - elasticsearch + ip: + - ipAddress + - name: kibana + dns: + - kibana + - localhost + ip: + - ipAddress + - name: logstash + dns: + - logstash + - localhost + ip: + - ipAddress diff --git a/src/test/resources/ssl/elasticsearch.yml b/src/test/resources/ssl/elasticsearch.yml index cd927813a..19834e529 100755 --- a/src/test/resources/ssl/elasticsearch.yml +++ b/src/test/resources/ssl/elasticsearch.yml @@ -19,12 +19,3 @@ xpack.security.transport.ssl.enabled: true xpack.security.transport.ssl.key: ssl/elasticsearch.key xpack.security.transport.ssl.certificate: ssl/elasticsearch.crt xpack.security.transport.ssl.certificate_authorities: [ "ssl/ca/ca.crt" ] - -# enable anonymous connections since setting passwords requires running a command -xpack.security.authc: - anonymous: - username: connect_user - roles: superuser - authz_exception: true - -