blob: 6d46863319f24145d7f28a9b7fb5a4c07cffb9b0 [file] [log] [blame]
/**
* 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.storm.elasticsearch.bolt;
import static java.util.Objects.requireNonNull;
import java.util.HashMap;
import java.util.Map;
import org.apache.http.entity.StringEntity;
import org.apache.storm.elasticsearch.common.DefaultEsTupleMapper;
import org.apache.storm.elasticsearch.common.EsConfig;
import org.apache.storm.elasticsearch.common.EsTupleMapper;
import org.apache.storm.task.OutputCollector;
import org.apache.storm.task.TopologyContext;
import org.apache.storm.topology.OutputFieldsDeclarer;
import org.apache.storm.tuple.Tuple;
/**
* Basic bolt for storing tuple to ES document.
*/
public class EsIndexBolt extends AbstractEsBolt {
private final EsTupleMapper tupleMapper;
/**
* EsIndexBolt constructor.
* @param esConfig Elasticsearch configuration containing node addresses {@link EsConfig}
*/
public EsIndexBolt(EsConfig esConfig) {
this(esConfig, new DefaultEsTupleMapper());
}
/**
* EsIndexBolt constructor.
* @param esConfig Elasticsearch configuration containing node addresses {@link EsConfig}
* @param tupleMapper Tuple to ES document mapper {@link EsTupleMapper}
*/
public EsIndexBolt(EsConfig esConfig, EsTupleMapper tupleMapper) {
super(esConfig);
this.tupleMapper = requireNonNull(tupleMapper);
}
@Override
public void prepare(Map<String, Object> map, TopologyContext topologyContext, OutputCollector outputCollector) {
super.prepare(map, topologyContext, outputCollector);
}
/**
* {@inheritDoc}
* Tuple should have relevant fields (source, index, type, id) for tupleMapper to extract ES document.
*/
@Override
public void process(Tuple tuple) {
try {
String source = tupleMapper.getSource(tuple);
String index = tupleMapper.getIndex(tuple);
String type = tupleMapper.getType(tuple);
String id = tupleMapper.getId(tuple);
Map<String, String> params = tupleMapper.getParams(tuple, new HashMap<>());
client.performRequest("put", getEndpoint(index, type, id), params, new StringEntity(source));
collector.ack(tuple);
} catch (Exception e) {
collector.reportError(e);
collector.fail(tuple);
}
}
@Override
public void declareOutputFields(OutputFieldsDeclarer outputFieldsDeclarer) {
}
}