New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[FLINK-4491] Handle index.number_of_shards in the ES connector #2790
Closed
Closed
Changes from all commits
Commits
Show all changes
5 commits
Select commit
Hold shift + click to select a range
bae4237
[FLINK-4491] Added index template and mappings creation
ddolzan d31c363
[FLINK-4491] Added javadoc to ElasticSearchHelper
ddolzan aed4a5b
[FLINK-4491] Added constants
ddolzan 474febb
Fixed job failure on bad documents (letting other threads to complete)
ddolzan 515d79f
Reverted some changes to check indexing problem
ddolzan File filter
Filter by extension
Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
186 changes: 186 additions & 0 deletions
186
...java/org/apache/flink/streaming/connectors/elasticsearch2/helper/ElasticSearchHelper.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,186 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one or more contributor license | ||
* agreements. See the NOTICE file distributed with this work for additional information regarding | ||
* copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the | ||
* "License"); you may not use this file except in compliance with the License. You may obtain a | ||
* copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software distributed under the License | ||
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express | ||
* or implied. See the License for the specific language governing permissions and limitations under | ||
* the License. | ||
*/ | ||
package org.apache.flink.streaming.connectors.elasticsearch2.helper; | ||
|
||
import com.google.common.collect.ImmutableList; | ||
|
||
import org.apache.flink.streaming.connectors.elasticsearch2.ElasticsearchSink; | ||
import org.elasticsearch.action.admin.indices.mapping.put.PutMappingRequest; | ||
import org.elasticsearch.action.admin.indices.template.put.PutIndexTemplateRequest; | ||
import org.elasticsearch.action.search.SearchResponse; | ||
import org.elasticsearch.client.Client; | ||
import org.elasticsearch.client.transport.TransportClient; | ||
import org.elasticsearch.cluster.metadata.IndexMetaData; | ||
import org.elasticsearch.cluster.node.DiscoveryNode; | ||
import org.elasticsearch.common.settings.Settings; | ||
import org.elasticsearch.common.transport.InetSocketTransportAddress; | ||
import org.elasticsearch.common.transport.TransportAddress; | ||
import org.elasticsearch.index.IndexNotFoundException; | ||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
||
import java.net.InetSocketAddress; | ||
import java.util.ArrayList; | ||
import java.util.List; | ||
import java.util.Map; | ||
|
||
/** | ||
* | ||
* This class manages the creation of index templates and index mapping on elasticsearch. | ||
* | ||
* <p> | ||
* Example: | ||
* | ||
* <pre>{@code | ||
* ElasticSearchHelper esHelper = new ElasticSearchHelper(config, transports); | ||
* | ||
* //Create an Index Template given a name and the json structure | ||
* esHelper.initTemplate(templateName, templateRequest); | ||
* | ||
* //Create an Index Mapping given the Index Name, DocType and the json structure | ||
* esHelper.initIndexMapping(indexName, docType, mappingsRequest); | ||
* | ||
* }</pre> | ||
* | ||
* <p> | ||
* The {@link Map} passed to the constructor is forwarded to Elasticsearch when | ||
* creating {@link TransportClient}. The config keys can be found in the | ||
* Elasticsearch documentation. An important setting is {@code cluster.name}, | ||
* this should be set to the name of the cluster that the sink should emit to. | ||
* | ||
*/ | ||
public class ElasticSearchHelper { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This class needs a javadoc that explains what this it can be used for. |
||
|
||
private static final Logger LOG = LoggerFactory.getLogger(ElasticsearchSink.class); | ||
|
||
private Client client; | ||
|
||
private final static int DEFAULT_INDEX_SHARDS = 2; | ||
private final static int DEFAULT_INDEX_REPLICAS = 0; | ||
|
||
/** | ||
* Creates a new ElasticSearchHelper that connects to the cluster using a TransportClient. | ||
* | ||
* @param userConfig The map of user settings that are passed when constructing the TransportClients | ||
* @param transportAddresses The Elasticsearch Nodes to which to connect using a {@code TransportClient} | ||
*/ | ||
public ElasticSearchHelper(Map<String, String> userConfig, List<InetSocketAddress> transportAddresses) { | ||
client = buildElasticsearchClient(userConfig, transportAddresses); | ||
} | ||
|
||
/** | ||
* Build a TransportClient to connect to the cluster. | ||
* | ||
* @param userConfig The map of user settings that are passed when constructing the TransportClients | ||
* @param transportAddresses The Elasticsearch Nodes to which to connect using a {@code TransportClient} | ||
* @return Initialized TransportClient | ||
*/ | ||
public static Client buildElasticsearchClient(Map<String, String> userConfig, | ||
List<InetSocketAddress> transportAddresses) { | ||
List<TransportAddress> transportNodes; | ||
transportNodes = new ArrayList<>(transportAddresses.size()); | ||
for (InetSocketAddress address : transportAddresses) { | ||
transportNodes.add(new InetSocketTransportAddress(address)); | ||
} | ||
|
||
Settings settings = Settings.settingsBuilder().put(userConfig).build(); | ||
|
||
TransportClient transportClient = TransportClient.builder().settings(settings).build(); | ||
for (TransportAddress transport : transportNodes) { | ||
transportClient.addTransportAddress(transport); | ||
} | ||
|
||
// verify that we actually are connected to a cluster | ||
ImmutableList<DiscoveryNode> nodes = ImmutableList.copyOf(transportClient.connectedNodes()); | ||
if (nodes.isEmpty()) { | ||
throw new RuntimeException("Client is not connected to any Elasticsearch nodes!"); | ||
} | ||
return transportClient; | ||
} | ||
|
||
/** | ||
* Create a new index template. | ||
* | ||
* @param templateName Name of the template to create | ||
* @param templateReq Json defining the index template | ||
*/ | ||
public void initTemplate(String templateName, String templateReq) throws Exception { | ||
// Check if the template is set | ||
if (templateReq != null && !templateReq.equals("")) { | ||
// Json deserialization | ||
PutIndexTemplateRequest request = new PutIndexTemplateRequest(templateName).source(templateReq); | ||
// Sending the request to elastic search | ||
sendIndexTemplateRequest(request); | ||
} | ||
} | ||
|
||
/** | ||
* Send the index template request to elasticsearch. | ||
* | ||
* @param indexTemplateRequest A valid index template request | ||
*/ | ||
public void sendIndexTemplateRequest(PutIndexTemplateRequest indexTemplateRequest) throws Exception { | ||
// Check if the template is set | ||
if (indexTemplateRequest != null) { | ||
// Sending the request to elastic search | ||
client.admin().indices().putTemplate(indexTemplateRequest).get(); | ||
} | ||
} | ||
|
||
/** | ||
* Create a new mapping for a document type for an index. | ||
* | ||
* @param indexName Index name where add the mapping | ||
* @param docType Document type of the mapping | ||
* @param mappingReq Json defining the index mapping | ||
*/ | ||
public void initIndexMapping(String indexName, String docType, String mappingReq) throws Exception { | ||
PutMappingRequest request=new PutMappingRequest(indexName).source(mappingReq).type(docType); | ||
|
||
// Put the mapping to the index | ||
sendIndexMappingRequest(request); | ||
LOG.debug("Updating mappings..."); | ||
} | ||
|
||
/** | ||
* Send the index mapping request to elasticsearch. | ||
* | ||
* @param mappingRequest A valid index mapping request | ||
*/ | ||
public void sendIndexMappingRequest(PutMappingRequest mappingRequest) throws Exception { | ||
// Check if the template is set | ||
if (mappingRequest != null) { | ||
try { | ||
// Check if the index exists | ||
SearchResponse response = client.prepareSearch(mappingRequest.indices()) | ||
.setTypes(mappingRequest.type()).get(); | ||
if (response != null) { | ||
LOG.debug("Index found, no need to create it..."); | ||
} | ||
} catch (IndexNotFoundException infe) { | ||
for (String indexName:mappingRequest.indices()){ | ||
// If the index does not exist, create it | ||
client.admin().indices().prepareCreate(indexName) | ||
.setSettings(Settings.builder().put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, DEFAULT_INDEX_SHARDS) | ||
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, DEFAULT_INDEX_REPLICAS)).execute().actionGet(); | ||
LOG.info("Index "+indexName+" not found, creating it..."); | ||
} | ||
} | ||
// Sending the request to elastic search | ||
client.admin().indices().putMapping(mappingRequest).get(); | ||
} | ||
} | ||
|
||
} |
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I would suggest to add a debug log statement as well logging the full stack trace.
Also, in the other connectors we have a flag that allows the user to control whether an error should be logged or fail the connector. I would suggest to add this here as well.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Do you mean to substitute the current line 263 with a LOG.DEBUG + stacktrace or leave the current line or to add another LOG.debug line where we log only the stacktrace? Currently the detail of the error is logged within afterBulk(). Should we modify that part?
How do you suggest to control the error behaviour ? Is it ok comething like
public static final String CONFIG_KEY_BULK_ERROR_POLICY = "bulk.error.policy";
that a user can set to "strict" or "lenient" in the userConfig object?