wonderdog 0.0.1

This diff represents the content of publicly available package versions that have been released to one of the supported registries. The information contained in this diff is provided for informational purposes only and reflects changes between package versions as they appear in their respective public registries.
Files changed (55) hide show
  1. data/.gitignore +49 -0
  2. data/.rspec +2 -0
  3. data/CHANGELOG.md +5 -0
  4. data/LICENSE.md +201 -0
  5. data/README.md +175 -0
  6. data/Rakefile +10 -0
  7. data/bin/estool +141 -0
  8. data/bin/estrus.rb +136 -0
  9. data/bin/wonderdog +93 -0
  10. data/config/elasticsearch-example.yml +227 -0
  11. data/config/elasticsearch.in.sh +52 -0
  12. data/config/logging.yml +43 -0
  13. data/config/more_settings.yml +60 -0
  14. data/config/run_elasticsearch-2.sh +42 -0
  15. data/config/ufo_config.json +12 -0
  16. data/lib/wonderdog.rb +14 -0
  17. data/lib/wonderdog/configuration.rb +25 -0
  18. data/lib/wonderdog/hadoop_invocation_override.rb +139 -0
  19. data/lib/wonderdog/index_and_mapping.rb +67 -0
  20. data/lib/wonderdog/timestamp.rb +43 -0
  21. data/lib/wonderdog/version.rb +3 -0
  22. data/notes/README-benchmarking.txt +272 -0
  23. data/notes/README-read_tuning.textile +74 -0
  24. data/notes/benchmarking-201011.numbers +0 -0
  25. data/notes/cluster_notes.md +17 -0
  26. data/notes/notes.txt +91 -0
  27. data/notes/pigstorefunc.pig +45 -0
  28. data/pom.xml +80 -0
  29. data/spec/spec_helper.rb +22 -0
  30. data/spec/support/driver_helper.rb +15 -0
  31. data/spec/support/integration_helper.rb +30 -0
  32. data/spec/wonderdog/hadoop_invocation_override_spec.rb +81 -0
  33. data/spec/wonderdog/index_and_type_spec.rb +73 -0
  34. data/src/main/java/com/infochimps/elasticsearch/ElasticSearchInputFormat.java +268 -0
  35. data/src/main/java/com/infochimps/elasticsearch/ElasticSearchOutputCommitter.java +39 -0
  36. data/src/main/java/com/infochimps/elasticsearch/ElasticSearchOutputFormat.java +283 -0
  37. data/src/main/java/com/infochimps/elasticsearch/ElasticSearchSplit.java +60 -0
  38. data/src/main/java/com/infochimps/elasticsearch/ElasticSearchStreamingInputFormat.java +231 -0
  39. data/src/main/java/com/infochimps/elasticsearch/ElasticSearchStreamingOutputCommitter.java +37 -0
  40. data/src/main/java/com/infochimps/elasticsearch/ElasticSearchStreamingOutputFormat.java +88 -0
  41. data/src/main/java/com/infochimps/elasticsearch/ElasticSearchStreamingRecordReader.java +176 -0
  42. data/src/main/java/com/infochimps/elasticsearch/ElasticSearchStreamingRecordWriter.java +171 -0
  43. data/src/main/java/com/infochimps/elasticsearch/ElasticSearchStreamingSplit.java +102 -0
  44. data/src/main/java/com/infochimps/elasticsearch/ElasticTest.java +108 -0
  45. data/src/main/java/com/infochimps/elasticsearch/hadoop/util/HadoopUtils.java +100 -0
  46. data/src/main/java/com/infochimps/elasticsearch/pig/ElasticSearchIndex.java +216 -0
  47. data/src/main/java/com/infochimps/elasticsearch/pig/ElasticSearchJsonIndex.java +235 -0
  48. data/src/main/java/com/infochimps/elasticsearch/pig/ElasticSearchStorage.java +355 -0
  49. data/test/foo.json +3 -0
  50. data/test/foo.tsv +3 -0
  51. data/test/test_dump.pig +19 -0
  52. data/test/test_json_loader.pig +21 -0
  53. data/test/test_tsv_loader.pig +16 -0
  54. data/wonderdog.gemspec +32 -0
  55. metadata +130 -0
@@ -0,0 +1,45 @@
1
+ --
2
+ -- Doesn't work at the moment, just some notes on how the storefunc might look.
3
+ --
4
+
5
+
6
+ --
7
+ -- Right now the ElasticSearchOutputFormat gets all its options from the
8
+ -- Job object. We can use the call to setStoreLocation in the storefunc
9
+ -- to set the required parameters. Need to make sure the following are
10
+ -- set:
11
+ --
12
+ -- wonderdog.index.name - should be set by the storefunc constructor
13
+ -- wonderdog.bulk.size - should be set by the storefunc constructor
14
+ -- wonderdog.field.names - should be set by the call to checkSchema
15
+ -- wonderdog.id.field - should be set by the storefunc constructor
16
+ -- wonderdog.object.type - should be set by the storefunc constructor
17
+ -- wonderdog.plugins.dir - should be set by call to setStoreLocation
18
+ -- wonderdog.config - should be set by call to setStoreLocation
19
+ --
20
+ -- FIXME: options used in the ElasticSearchOutputFormat should NOT be
21
+ -- namespaced with 'wonderdog'
22
+
23
+ %default INDEX 'es_index'
24
+ %default OBJ 'text_obj'
25
+
26
+
27
+ records = LOAD '$DATA' AS (text_field:chararray);
28
+ records_with_id = LOAD '$IDDATA' AS (id_field:int, text_field:chararray);
29
+
30
+ -- Here we would use the elasticsearch index name as the uri, pass in a
31
+ -- comma separated list of field names as the first arg, the id field
32
+ -- as the second arg and the bulk size as the third.
33
+ --
34
+ -- and so on.
35
+ STORE records INTO '$INDEX/$OBJ' USING ElasticSearchStorage('my_text_field', '-1', '1000');
36
+
37
+
38
+ -- but it would be really nice to duplicate what's in WonderDog.java in that,
39
+ -- should a bulk request fail, the failed records are written to hdfs. The
40
+ -- user should have some control of this. Also, it should be possible to generate
41
+ -- the field names directly from the pig schema? (We'd have to be VERY explicit in the
42
+ -- docs about this as it would be a point of headscratching/swearing...) In this
43
+ -- case we might have something like:
44
+ named_records = FOREACH records GENERATE text_field AS text_field_name;
45
+ STORE records INTO '/path/to/failed_requests' USING ElasticSearchStorage('$INDEX/$OBJ', '-1', '1000');
data/pom.xml ADDED
@@ -0,0 +1,80 @@
1
+ <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
2
+ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
3
+ <modelVersion>4.0.0</modelVersion>
4
+
5
+ <groupId>com.infochimps.elasticsearch</groupId>
6
+ <artifactId>wonderdog</artifactId>
7
+ <version>1.0-SNAPSHOT</version>
8
+ <packaging>jar</packaging>
9
+
10
+ <name>wonderdog</name>
11
+ <url>http://maven.apache.org</url>
12
+
13
+ <properties>
14
+ <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
15
+ </properties>
16
+
17
+ <dependencies>
18
+ <dependency>
19
+ <groupId>org.elasticsearch</groupId>
20
+ <artifactId>elasticsearch</artifactId>
21
+ <version>0.19.8</version>
22
+ </dependency>
23
+
24
+ <dependency>
25
+ <groupId>org.apache.hadoop</groupId>
26
+ <artifactId>hadoop-core</artifactId>
27
+ <version>0.20.2</version>
28
+ </dependency>
29
+
30
+ <dependency>
31
+ <groupId>org.apache.pig</groupId>
32
+ <artifactId>pig</artifactId>
33
+ <version>0.8.0</version>
34
+ </dependency>
35
+
36
+ <dependency>
37
+ <groupId>org.codehaus.jackson</groupId>
38
+ <artifactId>jackson-mapper-asl</artifactId>
39
+ <version>1.5.2</version>
40
+ </dependency>
41
+ </dependencies>
42
+
43
+ <build>
44
+ <plugins>
45
+ <plugin>
46
+ <groupId>org.apache.maven.plugins</groupId>
47
+ <artifactId>maven-compiler-plugin</artifactId>
48
+ <configuration>
49
+ <source>1.5</source>
50
+ <target>1.5</target>
51
+ </configuration>
52
+ </plugin>
53
+ </plugins>
54
+ </build>
55
+
56
+ <repositories>
57
+ <repository>
58
+ <id>com.cloudera</id>
59
+ <url>https://repository.cloudera.com/content/repositories/releases</url>
60
+ </repository>
61
+
62
+ <repository>
63
+ <id>sonatype-releases</id>
64
+ <url>http://oss.sonatype.org/content/repositories/releases</url>
65
+ </repository>
66
+
67
+ <repository>
68
+ <id>codehaus</id>
69
+ <url>http://repository.codehaus.org/org/codehaus</url>
70
+ <snapshots>
71
+ <enabled>true</enabled>
72
+ </snapshots>
73
+ <releases>
74
+ <enabled>true</enabled>
75
+ </releases>
76
+ </repository>
77
+
78
+ </repositories>
79
+
80
+ </project>
@@ -0,0 +1,22 @@
1
+ require 'wonderdog'
2
+ require 'wukong/spec_helpers'
3
+ require_relative('support/integration_helper')
4
+ require_relative('support/driver_helper')
5
+
6
+
7
+ RSpec.configure do |config|
8
+
9
+ config.before(:each) do
10
+ @orig_reg = Wukong.registry.show
11
+ end
12
+
13
+ config.after(:each) do
14
+ Wukong.registry.clear!
15
+ Wukong.registry.merge!(@orig_reg)
16
+ end
17
+
18
+ include Wukong::SpecHelpers
19
+ include Wukong::Elasticsearch::IntegrationHelper
20
+ include Wukong::Elasticsearch::DriverHelper
21
+ end
22
+
@@ -0,0 +1,15 @@
1
+ module Wukong
2
+ module Elasticsearch
3
+ module DriverHelper
4
+
5
+ def driver *args
6
+ params = Elasticsearch.configure(Hadoop.configure(Configliere::Param.new))
7
+ params.resolve!
8
+ params.merge!(args.pop) if args.last.is_a?(Hash)
9
+ Hadoop::Driver.new(params, *args)
10
+ end
11
+
12
+ end
13
+ end
14
+ end
15
+
@@ -0,0 +1,30 @@
1
+ module Wukong
2
+ module Elasticsearch
3
+ module IntegrationHelper
4
+
5
+ def root
6
+ @root ||= Pathname.new(File.expand_path('../../..', __FILE__))
7
+ end
8
+
9
+ def lib_dir
10
+ root.join('lib')
11
+ end
12
+
13
+ def bin_dir
14
+ root.join('bin')
15
+ end
16
+
17
+ def integration_env
18
+ {
19
+ "RUBYLIB" => [lib_dir.to_s, ENV["RUBYLIB"]].compact.join(':')
20
+ }
21
+ end
22
+
23
+ def integration_cwd
24
+ root.to_s
25
+ end
26
+
27
+ end
28
+ end
29
+ end
30
+
@@ -0,0 +1,81 @@
1
+ require 'spec_helper'
2
+
3
+ describe Wukong::Elasticsearch::HadoopInvocationOverride do
4
+
5
+ let(:no_es) { driver('regexp', 'count', input: '/tmp/input_file', output: '/tmp/output_file') }
6
+ let(:es_reader) { driver('regexp', 'count', input: 'es://the_index/the_map', output: '/tmp/output_file') }
7
+ let(:es_writer) { driver('regexp', 'count', input: '/tmp/input_file', output: 'es:///the_index/the_map') }
8
+ let(:es_complex) { driver('regexp', 'count', input: 'es://the_index/the_map', output: 'es:///the_index/the_map', es_query: '{"hi": "there"}', es_request_size: 1000, es_index_field: 'ID') }
9
+
10
+ context "not interacting with Elasticsearch" do
11
+ subject { no_es }
12
+ # input
13
+ its(:input_paths) { should == '/tmp/input_file' }
14
+ its(:hadoop_commandline) { should match(%r{-input.*/tmp/input_file}i) }
15
+
16
+ # output
17
+ its(:output_path) { should == '/tmp/output_file' }
18
+ its(:hadoop_commandline) { should match(%r{-output.*/tmp/output_file}i) }
19
+
20
+ # no elasticsearch anything
21
+ its(:hadoop_commandline) { should_not match(/elasticsearch/i) }
22
+ end
23
+
24
+ context "reading from Elasticsearch" do
25
+ subject { es_reader }
26
+
27
+ # input
28
+ its(:input_paths) { should match(%r{/user.*wukong.*the_index.*the_map}) }
29
+ its(:hadoop_commandline) { should match(/-inputformat.*elasticsearch/i) }
30
+ its(:hadoop_commandline) { should match(%r{-input.*/user.*wukong.*the_index.*the_map}i) }
31
+ its(:hadoop_commandline) { should match(/-D\s+elasticsearch\.input\.index.*the_index/i) }
32
+ its(:hadoop_commandline) { should match(/-D\s+elasticsearch\.input\.map.*the_map/i) }
33
+
34
+ # output
35
+ its(:output_path) { should == '/tmp/output_file' }
36
+ its(:hadoop_commandline) { should_not match(/-outputformat/i) }
37
+ its(:hadoop_commandline) { should match(%r{-output.*/tmp/output_file}i) }
38
+ its(:hadoop_commandline) { should_not match(/-D\s+elasticsearch\.output/i) }
39
+ end
40
+
41
+ context "writing to Elasticsearch" do
42
+ subject { es_writer }
43
+
44
+ # input
45
+ its(:input_paths) { should == '/tmp/input_file' }
46
+ its(:hadoop_commandline) { should_not match(/-inputformat/i) }
47
+ its(:hadoop_commandline) { should match(%r{-input.*/tmp/input_file}i) }
48
+ its(:hadoop_commandline) { should_not match(/-D\s+elasticsearch\.input/i) }
49
+
50
+ # output
51
+ its(:output_path) { should match(%r{/user.*wukong.*the_index.*the_map}) }
52
+ its(:hadoop_commandline) { should match(/-outputformat.*elasticsearch/i) }
53
+ its(:hadoop_commandline) { should match(%r{-output.*/user.*wukong.*the_index.*the_map}i) }
54
+ its(:hadoop_commandline) { should match(/-D\s+elasticsearch\.output\.index.*the_index/i) }
55
+ its(:hadoop_commandline) { should match(/-D\s+elasticsearch\.output\.map.*the_map/i) }
56
+ end
57
+
58
+ context "reading and writing with many options" do
59
+ subject { es_complex }
60
+
61
+ # input
62
+ its(:input_paths) { should match(%r{/user.*wukong.*the_index.*the_map}) }
63
+ its(:hadoop_commandline) { should match(/-inputformat.*elasticsearch/i) }
64
+ its(:hadoop_commandline) { should match(%r{-input.*/user.*wukong.*the_index.*the_map}i) }
65
+ its(:hadoop_commandline) { should match(/-D\s+elasticsearch\.input\.index.*the_index/i) }
66
+ its(:hadoop_commandline) { should match(/-D\s+elasticsearch\.input\.map.*the_map/i) }
67
+
68
+ # output
69
+ its(:output_path) { should match(%r{/user.*wukong.*the_index.*the_map}) }
70
+ its(:hadoop_commandline) { should match(/-outputformat.*elasticsearch/i) }
71
+ its(:hadoop_commandline) { should match(%r{-output.*/user.*wukong.*the_index.*the_map}i) }
72
+ its(:hadoop_commandline) { should match(/-D\s+elasticsearch\.output\.index.*the_index/i) }
73
+ its(:hadoop_commandline) { should match(/-D\s+elasticsearch\.output\.map.*the_map/i) }
74
+
75
+ # options
76
+ its(:hadoop_commandline) { should match(/-D\s+elasticsearch\.input\.query.*hi.*there/i) }
77
+ its(:hadoop_commandline) { should match(/-D\s+elasticsearch\.input\.request_size.*1000/i) }
78
+ its(:hadoop_commandline) { should match(/-D\s+elasticsearch\.output\.index\.field.*ID/i) }
79
+ end
80
+
81
+ end
@@ -0,0 +1,73 @@
1
+ require 'spec_helper'
2
+
3
+ describe Wukong::Elasticsearch::IndexAndMapping do
4
+
5
+ subject { Wukong::Elasticsearch::IndexAndMapping }
6
+
7
+ let(:filesystem_path) { '/some/path' }
8
+ let(:filesystem_paths) { '/some/path,/some/other/path' }
9
+
10
+ let(:hdfs_path) { 'hdfs://some/hdfs/path' }
11
+ let(:hdfs_paths) { 'hdfs://some/hdfs/path,hdfs://some/other/hdfs/path' }
12
+
13
+ let(:es_index_and_mapping) { 'es://index/mapping' }
14
+ let(:es_indices_and_mapping) { 'es://index1,index2/mapping' }
15
+ let(:es_index_and_mappings) { 'es://index/mapping1,mapping2' }
16
+ let(:es_indices_and_mappings) { 'es://index1,index2/mapping1,mapping2' }
17
+
18
+ fails = %w[filesystem_path filesystem_paths hdfs_path hdfs_paths]
19
+ passes = %w[es_index_and_mapping es_indices_and_mapping es_index_and_mappings es_indices_and_mappings]
20
+
21
+ context 'recognizing possible es://index/mapping specifications' do
22
+ fails.each do |name|
23
+ it "doesn't recognize a #{name}" do
24
+ subject.matches?(self.send(name)).should be_false
25
+ end
26
+ end
27
+ passes.each do |name|
28
+ it "recognizes a #{name}" do
29
+ subject.matches?(self.send(name)).should be_true
30
+ end
31
+ end
32
+ end
33
+
34
+ context "parsing es://index/mapping specifications" do
35
+ fails.each do |name|
36
+ it "raises an error on a #{name}" do
37
+ lambda { subject.new(self.send(name)) }.should raise_error(Wukong::Error, /not an elasticsearch.*index\/mapping/i)
38
+ end
39
+ end
40
+
41
+ it "raises an error on a specification with too many parts" do
42
+ lambda { subject.new('es://index/mapping/extra') }.should raise_error(Wukong::Error, /not an elasticsearch.*index\/mapping/i)
43
+ end
44
+
45
+ it "raises an error on a specification with too few parts" do
46
+ lambda { subject.new('es://') }.should raise_error(Wukong::Error, /not an elasticsearch.*index\/mapping/i)
47
+ end
48
+
49
+ context "on an index and mapping" do
50
+ subject { Wukong::Elasticsearch::IndexAndMapping.new(es_index_and_mapping) }
51
+ its(:index) { should == 'index' }
52
+ its(:mapping) { should == 'mapping' }
53
+ end
54
+ context "on indices and a mapping" do
55
+ subject { Wukong::Elasticsearch::IndexAndMapping.new(es_indices_and_mapping) }
56
+ its(:index) { should == 'index1,index2' }
57
+ its(:mapping) { should == 'mapping' }
58
+ end
59
+ context "on an index and mappings" do
60
+ subject { Wukong::Elasticsearch::IndexAndMapping.new(es_index_and_mappings) }
61
+ its(:index) { should == 'index' }
62
+ its(:mapping) { should == 'mapping1,mapping2' }
63
+ end
64
+ context "on indices and mappings" do
65
+ subject { Wukong::Elasticsearch::IndexAndMapping.new(es_indices_and_mappings) }
66
+ its(:index) { should == 'index1,index2' }
67
+ its(:mapping) { should == 'mapping1,mapping2' }
68
+ end
69
+
70
+
71
+ end
72
+
73
+ end
@@ -0,0 +1,268 @@
1
+ package com.infochimps.elasticsearch;
2
+
3
+ import java.io.IOException;
4
+ import java.util.Map;
5
+ import java.util.List;
6
+ import java.util.ArrayList;
7
+ import java.util.Iterator;
8
+
9
+ import org.apache.commons.logging.Log;
10
+ import org.apache.commons.logging.LogFactory;
11
+
12
+ import org.apache.hadoop.io.*;
13
+ import org.apache.hadoop.conf.Configurable;
14
+ import org.apache.hadoop.conf.Configuration;
15
+ import org.apache.hadoop.mapreduce.InputFormat;
16
+ import org.apache.hadoop.mapreduce.RecordReader;
17
+ import org.apache.hadoop.mapreduce.InputSplit;
18
+ import org.apache.hadoop.mapreduce.JobContext;
19
+ import org.apache.hadoop.mapreduce.TaskAttemptContext;
20
+
21
+ import com.infochimps.elasticsearch.hadoop.util.HadoopUtils;
22
+
23
+ import org.elasticsearch.node.Node;
24
+ import org.elasticsearch.node.NodeBuilder;
25
+ import org.elasticsearch.client.Client;
26
+ import org.elasticsearch.client.Requests;
27
+ import org.elasticsearch.search.SearchHit;
28
+ import org.elasticsearch.action.search.SearchResponse;
29
+ import org.elasticsearch.action.search.SearchType;
30
+ import org.elasticsearch.index.query.FilterBuilders.*;
31
+ import org.elasticsearch.index.query.QueryBuilders;
32
+
33
+ /**
34
+
35
+ A Hadoop InputFormat to read data from an Elasticsearch index. The RecordReader
36
+ divulges records where the key is the record id in elasticsearch and the value
37
+ is a json string of the (source) record contents.
38
+
39
+ */
40
+ public class ElasticSearchInputFormat extends InputFormat<Text, Text> implements Configurable {
41
+
42
+ static Log LOG = LogFactory.getLog(ElasticSearchInputFormat.class);
43
+ private Configuration conf = null;
44
+
45
+ private Node node;
46
+ private Client client;
47
+
48
+ private Integer requestSize;
49
+ private Long numHits;
50
+ private Long numSplits;
51
+ private Long numSplitRecords;
52
+ private String indexName;
53
+ private String objType;
54
+ private String queryString;
55
+
56
+ private static final String ES_REQUEST_SIZE = "elasticsearch.request.size"; // number of records to fetch at one time
57
+ private static final String ES_NUM_SPLITS = "elasticsearch.num.input.splits"; // number of hadoop map tasks to launch
58
+ private static final String ES_QUERY_STRING = "elasticsearch.query.string";
59
+
60
+ private static final String ES_CONFIG_NAME = "elasticsearch.yml";
61
+ private static final String ES_PLUGINS_NAME = "plugins";
62
+ private static final String ES_INDEX_NAME = "elasticsearch.index.name";
63
+ private static final String ES_OBJECT_TYPE = "elasticsearch.object.type";
64
+ private static final String ES_CONFIG = "es.config";
65
+ private static final String ES_PLUGINS = "es.path.plugins";
66
+ private static final String SLASH = "/";
67
+
68
+ public RecordReader<Text,Text> createRecordReader(InputSplit inputSplit,
69
+ TaskAttemptContext context) {
70
+ return new ElasticSearchRecordReader();
71
+ }
72
+
73
+ /**
74
+ The number of splits is specified in the Hadoop configuration object.
75
+ */
76
+ public List<InputSplit> getSplits(JobContext context) {
77
+ setConf(context.getConfiguration());
78
+ List<InputSplit> splits = new ArrayList<InputSplit>(numSplits.intValue());
79
+ for(int i = 0; i < numSplits; i++) {
80
+ Long size = (numSplitRecords == 1) ? 1 : numSplitRecords-1;
81
+ splits.add(new ElasticSearchSplit(queryString, i*numSplitRecords, size));
82
+ }
83
+ if (numHits % numSplits > 0) splits.add(new ElasticSearchSplit(queryString, numSplits*numSplitRecords, numHits % numSplits - 1));
84
+ LOG.info("Created ["+splits.size()+"] splits for ["+numHits+"] hits");
85
+ return splits;
86
+ }
87
+
88
+ /**
89
+ Sets the configuration object, opens a connection to elasticsearch, and
90
+ initiates the initial search request.
91
+ */
92
+ @Override
93
+ public void setConf(Configuration configuration) {
94
+ this.conf = configuration;
95
+ this.indexName = conf.get(ES_INDEX_NAME);
96
+ this.objType = conf.get(ES_OBJECT_TYPE);
97
+ this.requestSize = Integer.parseInt(conf.get(ES_REQUEST_SIZE));
98
+ this.numSplits = Long.parseLong(conf.get(ES_NUM_SPLITS));
99
+ this.queryString = conf.get(ES_QUERY_STRING);
100
+
101
+ //
102
+ // Need to ensure that this is set in the hadoop configuration so we can
103
+ // instantiate a local client. The reason is that no files are in the
104
+ // distributed cache when this is called.
105
+ //
106
+ System.setProperty(ES_CONFIG, conf.get(ES_CONFIG));
107
+ System.setProperty(ES_PLUGINS, conf.get(ES_PLUGINS));
108
+
109
+ start_embedded_client();
110
+
111
+ initiate_search();
112
+ }
113
+
114
+ @Override
115
+ public Configuration getConf() {
116
+ return conf;
117
+ }
118
+
119
+ /**
120
+ Starts an embedded elasticsearch client (ie. data = false)
121
+ */
122
+ private void start_embedded_client() {
123
+ LOG.info("Starting embedded elasticsearch client ...");
124
+ this.node = NodeBuilder.nodeBuilder().client(true).node();
125
+ this.client = node.client();
126
+ }
127
+
128
+ private void initiate_search() {
129
+ SearchResponse response = client.prepareSearch(indexName)
130
+ .setTypes(objType)
131
+ .setSearchType(SearchType.COUNT)
132
+ .setQuery(QueryBuilders.queryString(queryString))
133
+ .setSize(requestSize)
134
+ .execute()
135
+ .actionGet();
136
+ this.numHits = response.hits().totalHits();
137
+ if(numSplits > numHits) numSplits = numHits; // This could be bad
138
+ this.numSplitRecords = (numHits/numSplits);
139
+ }
140
+
141
+ protected class ElasticSearchRecordReader extends RecordReader<Text, Text> {
142
+
143
+ private Node node;
144
+ private Client client;
145
+
146
+ private String indexName;
147
+ private String objType;
148
+ private Long numSplitRecords;
149
+ private Integer requestSize;
150
+ private Text currentKey;
151
+ private Text currentValue;
152
+ private Integer recordsRead;
153
+ private Iterator<SearchHit> hitsItr = null;
154
+
155
+
156
+ private String queryString;
157
+ private Long from;
158
+ private Long recsToRead;
159
+
160
+ public ElasticSearchRecordReader() {
161
+ }
162
+
163
+ public void initialize(InputSplit split, TaskAttemptContext context) throws IOException {
164
+ Configuration conf = context.getConfiguration();
165
+ this.indexName = conf.get(ES_INDEX_NAME);
166
+ this.objType = conf.get(ES_OBJECT_TYPE);
167
+ LOG.info("Initializing elasticsearch record reader on index ["+indexName+"] and object type ["+objType+"]");
168
+
169
+ //
170
+ // Fetches elasticsearch.yml and the plugins directory from the distributed cache
171
+ //
172
+ try {
173
+ String taskConfigPath = HadoopUtils.fetchFileFromCache(ES_CONFIG_NAME, conf);
174
+ LOG.info("Using ["+taskConfigPath+"] as es.config");
175
+ String taskPluginsPath = HadoopUtils.fetchArchiveFromCache(ES_PLUGINS_NAME, conf);
176
+ LOG.info("Using ["+taskPluginsPath+"] as es.plugins.dir");
177
+ System.setProperty(ES_CONFIG, taskConfigPath);
178
+ System.setProperty(ES_PLUGINS, taskPluginsPath+SLASH+ES_PLUGINS_NAME);
179
+ } catch (Exception e) {
180
+ throw new RuntimeException(e);
181
+ }
182
+
183
+ queryString = ((ElasticSearchSplit)split).getQueryString();
184
+ from = ((ElasticSearchSplit)split).getFrom();
185
+ recsToRead = ((ElasticSearchSplit)split).getSize();
186
+
187
+ LOG.info("elasticsearch record reader: query ["+queryString+"], from ["+from+"], size ["+recsToRead+"]");
188
+ start_embedded_client();
189
+ recordsRead = 0;
190
+ }
191
+
192
+ /**
193
+ Starts an embedded elasticsearch client (ie. data = false)
194
+ */
195
+ private void start_embedded_client() {
196
+ LOG.info("Starting embedded elasticsearch client ...");
197
+ this.node = NodeBuilder.nodeBuilder().client(true).node();
198
+ this.client = node.client();
199
+ }
200
+
201
+ private Iterator<SearchHit> fetchNextHits() {
202
+ SearchResponse response = client.prepareSearch(indexName)
203
+ .setTypes(objType)
204
+ .setFrom(from.intValue())
205
+ .setSize(recsToRead.intValue())
206
+ .setQuery(QueryBuilders.queryString(queryString))
207
+ .execute()
208
+ .actionGet();
209
+ return response.hits().iterator();
210
+ }
211
+
212
+ @Override
213
+ public boolean nextKeyValue() throws IOException {
214
+ if (hitsItr!=null) {
215
+ if (recordsRead < recsToRead) {
216
+ if (hitsItr.hasNext()) {
217
+ SearchHit hit = hitsItr.next();
218
+ currentKey = new Text(hit.id());
219
+ currentValue = new Text(hit.sourceAsString());
220
+ recordsRead += 1;
221
+ return true;
222
+ }
223
+ } else {
224
+ hitsItr = null;
225
+ }
226
+ } else {
227
+ if (recordsRead < recsToRead) {
228
+ hitsItr = fetchNextHits();
229
+ if (hitsItr.hasNext()) {
230
+ SearchHit hit = hitsItr.next();
231
+ currentKey = new Text(hit.id());
232
+ currentValue = new Text(hit.sourceAsString());
233
+ recordsRead += 1;
234
+ return true;
235
+ }
236
+ }
237
+ }
238
+ return false;
239
+ }
240
+
241
+ @Override
242
+ public Text getCurrentKey() {
243
+ return currentKey;
244
+ }
245
+
246
+ @Override
247
+ public Text getCurrentValue() {
248
+ return currentValue;
249
+ }
250
+
251
+ @Override
252
+ public float getProgress() throws IOException {
253
+ return 0;
254
+ }
255
+
256
+ @Override
257
+ public void close() throws IOException {
258
+ LOG.info("Closing record reader");
259
+ client.close();
260
+ LOG.info("Client is closed");
261
+ if (node != null) {
262
+ node.close();
263
+ }
264
+ LOG.info("Record reader closed.");
265
+ }
266
+
267
+ }
268
+ }