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,102 @@
1
+ package com.infochimps.elasticsearch;
2
+
3
+ import java.io.IOException;
4
+ import java.io.DataInput;
5
+ import java.io.DataOutput;
6
+
7
+ import org.apache.hadoop.io.Text;
8
+ import org.apache.hadoop.io.Writable;
9
+ import org.apache.hadoop.mapred.InputSplit;
10
+
11
+ import org.elasticsearch.search.Scroll;
12
+
13
+ import org.elasticsearch.client.Client;
14
+ import org.elasticsearch.action.search.SearchRequestBuilder;
15
+ import org.elasticsearch.action.search.SearchScrollRequestBuilder;
16
+ import org.elasticsearch.action.search.SearchType;
17
+
18
+ public class ElasticSearchStreamingSplit implements InputSplit, Writable {
19
+
20
+ private String indexName;
21
+ private String mappingName;
22
+ private Integer numSplits;
23
+ private String queryJSON;
24
+ private Long numHits;
25
+ private Integer from;
26
+ private Integer size;
27
+
28
+ public ElasticSearchStreamingSplit() {
29
+ }
30
+
31
+ public ElasticSearchStreamingSplit(String indexName , String mappingName, Integer numSplits, String queryJSON, Long numHits, Integer from, Integer size) {
32
+ this.indexName = indexName;
33
+ this.mappingName = mappingName;
34
+ this.numSplits = numSplits;
35
+ this.queryJSON = queryJSON;
36
+ this.numHits = numHits;
37
+ this.from = from;
38
+ this.size = size;
39
+ }
40
+
41
+ public String getSummary() {
42
+ Integer thisSplitNum = (int) (((long) from) / (numHits / ((long) numSplits)));
43
+ return "ElasticSearch input split "+String.valueOf(thisSplitNum + 1)+"/"+String.valueOf(numSplits)+" with "+String.valueOf(size)+" records from /"+indexName+"/"+mappingName;
44
+ }
45
+
46
+ public Integer getSize() {
47
+ return size;
48
+ }
49
+
50
+ public boolean hasQuery() {
51
+ return queryJSON != null && queryJSON.length() > 0;
52
+ }
53
+
54
+ public SearchRequestBuilder initialScrollRequest(Client client, Scroll scroll, Integer requestSize) {
55
+ SearchRequestBuilder request = client.prepareSearch(indexName).setSearchType(SearchType.SCAN).setScroll(scroll);
56
+ if (mappingName != null && mappingName.length() > 0) {
57
+ request.setTypes(mappingName);
58
+ }
59
+ request.setFrom((int) from);
60
+ request.setSize(requestSize);
61
+ if (hasQuery()) {
62
+ request.setQuery(queryJSON);
63
+ }
64
+ return request;
65
+ }
66
+
67
+ public SearchScrollRequestBuilder scrollRequest(Client client, Scroll scroll, String scrollId) {
68
+ return client.prepareSearchScroll(scrollId).setScroll(scroll);
69
+ }
70
+
71
+ @Override
72
+ public String[] getLocations() {
73
+ return new String[] {};
74
+ }
75
+
76
+ @Override
77
+ public long getLength() {
78
+ return 0;
79
+ }
80
+
81
+ @Override
82
+ public void readFields(DataInput in) throws IOException {
83
+ this.indexName = Text.readString(in);
84
+ this.mappingName = Text.readString(in);
85
+ this.numSplits = in.readInt();
86
+ this.queryJSON = Text.readString(in);
87
+ this.numHits = in.readLong();
88
+ this.from = in.readInt();
89
+ this.size = in.readInt();
90
+ }
91
+
92
+ @Override
93
+ public void write(DataOutput out) throws IOException {
94
+ Text.writeString(out, indexName);
95
+ Text.writeString(out, mappingName);
96
+ out.writeInt(numSplits);
97
+ Text.writeString(out, queryJSON);
98
+ out.writeLong(numHits);
99
+ out.writeInt(from);
100
+ out.writeInt(size);
101
+ }
102
+ }
@@ -0,0 +1,108 @@
1
+ package com.infochimps.elasticsearch;
2
+
3
+ import java.io.IOException;
4
+ import java.util.ArrayList;
5
+ import java.util.List;
6
+ import java.util.concurrent.atomic.AtomicLong;
7
+ import java.util.Random;
8
+ import java.util.Map;
9
+ import java.util.HashMap;
10
+
11
+ import org.apache.commons.logging.Log;
12
+ import org.apache.commons.logging.LogFactory;
13
+
14
+ import org.apache.hadoop.mapreduce.Job;
15
+ import org.apache.hadoop.mapreduce.Mapper;
16
+ import org.apache.hadoop.mapreduce.Counter;
17
+ import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
18
+ import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
19
+ import org.apache.hadoop.fs.Path;
20
+ import org.apache.hadoop.util.Tool;
21
+ import org.apache.hadoop.util.ToolRunner;
22
+ import org.apache.hadoop.util.GenericOptionsParser;
23
+ import org.apache.hadoop.conf.Configured;
24
+ import org.apache.hadoop.conf.Configuration;
25
+ import org.apache.hadoop.io.LongWritable;
26
+ import org.apache.hadoop.io.NullWritable;
27
+ import org.apache.hadoop.io.Text;
28
+ import org.apache.hadoop.io.MapWritable;
29
+
30
+ import org.elasticsearch.common.xcontent.XContentBuilder;
31
+ import org.elasticsearch.common.xcontent.XContentFactory;
32
+ import org.elasticsearch.node.Node;
33
+ import org.elasticsearch.node.NodeBuilder;
34
+ import org.elasticsearch.client.Client;
35
+ import org.elasticsearch.client.Requests;
36
+ import org.elasticsearch.action.bulk.BulkRequestBuilder;
37
+ import org.elasticsearch.indices.IndexAlreadyExistsException;
38
+ import org.elasticsearch.action.bulk.BulkResponse;
39
+ import org.elasticsearch.ExceptionsHelper;
40
+
41
+
42
+ //
43
+ // Simple one-hop bulk indexing hadoop job for elasticsearch. It accepts
44
+ // tsv documents, creates batch index requests, and sends records directly
45
+ // to the elasticsearch data node that's going to actually index them.
46
+ //
47
+ public class ElasticTest extends Configured implements Tool {
48
+
49
+ private final static Log LOG = LogFactory.getLog(ElasticTest.class);
50
+
51
+ public static class IndexMapper extends Mapper<LongWritable, Text, NullWritable, MapWritable> {
52
+
53
+ private String[] fieldNames;
54
+ public void map(LongWritable key, Text value, Context context) throws IOException, InterruptedException {
55
+ String[] fields = value.toString().split("\t");
56
+ MapWritable record = new MapWritable();
57
+ for (int i = 0; i < fields.length; i++) {
58
+ if (i < fieldNames.length) {
59
+ record.put(new Text(fieldNames[i]), new Text(fields[i]));
60
+ }
61
+ }
62
+ context.write(NullWritable.get(), record);
63
+ }
64
+
65
+ //
66
+ // Called once at the beginning of the map task. Sets up the indexing job.
67
+ //
68
+ protected void setup(org.apache.hadoop.mapreduce.Mapper.Context context) throws IOException, InterruptedException {
69
+ Configuration conf = context.getConfiguration();
70
+ this.fieldNames = conf.get("wonderdog.field.names").split(",");
71
+ }
72
+
73
+ }
74
+
75
+ public int run(String[] args) throws Exception {
76
+ Job job = new Job(getConf());
77
+ job.setJarByClass(ElasticTest.class);
78
+ job.setJobName("ElasticTest");
79
+ job.setMapperClass(IndexMapper.class);
80
+ job.setNumReduceTasks(0);
81
+ job.setOutputKeyClass(NullWritable.class);
82
+ job.setOutputValueClass(MapWritable.class);
83
+ job.setOutputFormatClass(ElasticSearchOutputFormat.class);
84
+
85
+ List<String> other_args = new ArrayList<String>();
86
+ for (int i=0; i < args.length; ++i) {
87
+ System.out.println(args[i]);
88
+ other_args.add(args[i]);
89
+ }
90
+ // Here we need _both_ an input path and an output path.
91
+ // Output stores failed records so they can be re-indexed
92
+ FileInputFormat.setInputPaths(job, new Path(other_args.get(0)));
93
+ FileOutputFormat.setOutputPath(job, new Path(other_args.get(1)));
94
+
95
+ try {
96
+ job.waitForCompletion(true);
97
+ }
98
+ catch (IOException e) {
99
+ throw new RuntimeException(e);
100
+ }
101
+ return 0;
102
+ }
103
+
104
+ public static void main(String[] args) throws Exception {
105
+ int res = ToolRunner.run(new Configuration(), new ElasticTest(), args);
106
+ System.exit(res);
107
+ }
108
+ }
@@ -0,0 +1,100 @@
1
+ package com.infochimps.elasticsearch.hadoop.util;
2
+
3
+ import java.io.File;
4
+
5
+ import java.io.IOException;
6
+ import java.io.FileNotFoundException;
7
+
8
+ import org.apache.hadoop.fs.FileSystem;
9
+ import org.apache.hadoop.fs.FileStatus;
10
+ import org.apache.hadoop.fs.Path;
11
+ import org.apache.hadoop.conf.Configuration;
12
+ import org.apache.hadoop.filecache.DistributedCache;
13
+
14
+ public class HadoopUtils {
15
+
16
+ /**
17
+ Upload a local file to the cluster
18
+ */
19
+ public static void uploadLocalFile(Path localsrc, Path hdfsdest, Configuration conf) throws IOException {
20
+ FileSystem fs = FileSystem.get(conf);
21
+ if (fs.exists(hdfsdest) && fs.getFileStatus(hdfsdest).isDir()) {
22
+ fs.delete(hdfsdest, true);
23
+ }
24
+ fs.copyFromLocalFile(false, true, localsrc, hdfsdest);
25
+ }
26
+
27
+
28
+ /**
29
+ Upload a local file to the cluster, if it's newer or nonexistent
30
+ */
31
+ public static void uploadLocalFileIfChanged(Path localsrc, Path hdfsdest, Configuration conf) throws IOException {
32
+ long l_time = new File(localsrc.toUri()).lastModified();
33
+ try {
34
+ long h_time = FileSystem.get(conf).getFileStatus(hdfsdest).getModificationTime();
35
+ if ( l_time > h_time ) {
36
+ uploadLocalFile(localsrc, hdfsdest, conf);
37
+ }
38
+ }
39
+ catch (FileNotFoundException e) {
40
+ uploadLocalFile(localsrc, hdfsdest, conf);
41
+ }
42
+ }
43
+
44
+
45
+ /**
46
+ Fetches a file with the basename specified from the distributed cache. Returns null if no file is found
47
+ */
48
+ public static String fetchFileFromCache(String basename, Configuration conf) throws IOException {
49
+ Path[] cacheFiles = DistributedCache.getLocalCacheFiles(conf);
50
+ if (cacheFiles != null && cacheFiles.length > 0) {
51
+ for (Path cacheFile : cacheFiles) {
52
+ if (cacheFile.getName().equals(basename)) {
53
+ return cacheFile.toString();
54
+ }
55
+ }
56
+ }
57
+ return null;
58
+ }
59
+
60
+ /**
61
+ Fetches a file with the basename specified from the distributed cache. Returns null if no file is found
62
+ */
63
+ public static String fetchArchiveFromCache(String basename, Configuration conf) throws IOException {
64
+ Path[] cacheArchives = DistributedCache.getLocalCacheArchives(conf);
65
+ if (cacheArchives != null && cacheArchives.length > 0) {
66
+ for (Path cacheArchive : cacheArchives) {
67
+ if (cacheArchive.getName().equals(basename)) {
68
+ return cacheArchive.toString();
69
+ }
70
+ }
71
+ }
72
+ return null;
73
+ }
74
+
75
+ /**
76
+ Takes a path on the hdfs and ships it in the distributed cache if it is not already in the distributed cache
77
+ */
78
+ public static void shipFileIfNotShipped(Path hdfsPath, Configuration conf) throws IOException {
79
+ if (fetchFileFromCache(hdfsPath.getName(), conf) == null) {
80
+ try {
81
+ DistributedCache.addCacheFile(hdfsPath.toUri(), conf);
82
+ } catch (Exception e) {
83
+ throw new RuntimeException(e);
84
+ }
85
+ }
86
+ }
87
+
88
+ /**
89
+ Takes a path on the hdfs and ships it in the distributed cache if it is not already in the distributed cache
90
+ */
91
+ public static void shipArchiveIfNotShipped(Path hdfsPath, Configuration conf) throws IOException {
92
+ if (fetchArchiveFromCache(hdfsPath.getName(), conf) == null) {
93
+ try {
94
+ DistributedCache.addCacheArchive(hdfsPath.toUri(), conf);
95
+ } catch (Exception e) {
96
+ throw new RuntimeException(e);
97
+ }
98
+ }
99
+ }
100
+ }
@@ -0,0 +1,216 @@
1
+ package com.infochimps.elasticsearch.pig;
2
+
3
+ import java.io.ByteArrayOutputStream;
4
+ import java.io.DataOutputStream;
5
+ import java.io.IOException;
6
+ import java.util.Arrays;
7
+ import java.util.List;
8
+ import java.util.Map;
9
+ import java.util.HashMap;
10
+ import java.util.Properties;
11
+ import java.net.URI;
12
+
13
+ import org.apache.commons.logging.Log;
14
+ import org.apache.commons.logging.LogFactory;
15
+ import org.apache.hadoop.conf.Configuration;
16
+ import org.apache.hadoop.fs.Path;
17
+ import org.apache.hadoop.io.NullWritable;
18
+ import org.apache.hadoop.io.Text;
19
+ import org.apache.hadoop.io.MapWritable;
20
+ import org.apache.hadoop.mapreduce.InputFormat;
21
+ import org.apache.hadoop.mapreduce.Job;
22
+ import org.apache.hadoop.mapreduce.OutputFormat;
23
+ import org.apache.hadoop.mapreduce.RecordReader;
24
+ import org.apache.hadoop.mapreduce.RecordWriter;
25
+ import org.apache.hadoop.filecache.DistributedCache;
26
+
27
+ import org.apache.pig.StoreFunc;
28
+ import org.apache.pig.ResourceSchema;
29
+ import org.apache.pig.ResourceSchema.ResourceFieldSchema;
30
+ import org.apache.pig.StoreFuncInterface;
31
+ import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigSplit;
32
+ import org.apache.pig.builtin.Utf8StorageConverter;
33
+ import org.apache.pig.data.DataBag;
34
+ import org.apache.pig.data.DataByteArray;
35
+ import org.apache.pig.data.DataType;
36
+ import org.apache.pig.data.Tuple;
37
+ import org.apache.pig.data.TupleFactory;
38
+ import org.apache.pig.impl.logicalLayer.FrontendException;
39
+ import org.apache.pig.impl.util.Utils;
40
+ import org.apache.pig.impl.util.UDFContext;
41
+
42
+ import com.infochimps.elasticsearch.hadoop.util.HadoopUtils;
43
+ import com.infochimps.elasticsearch.ElasticSearchOutputFormat;
44
+
45
+ /**
46
+ Pig storefunc for Elastic Search. Takes tuples of any primitive type, converts them
47
+ to strings, and indexes them.
48
+
49
+ USAGE:
50
+
51
+ STORE records INTO ElasticSearchIndex();
52
+ STORE records INTO ElasticSearchIndex(idField, bulkSize);
53
+ STORE records INTO ElasticSearchIndex(idField, bulkSize, esConfig);
54
+ STORE records INTO ElasticSearchIndex(idField, bulkSize, esConfig, esPlugins);
55
+
56
+ where:
57
+
58
+ idField = Which field of the record to use as the record id. If none is passed in
59
+ then the record is assumed to have no id.
60
+ bulkSize = Number of records for ElasticSearchOutputFormat to batch up before sending
61
+ a bulk index request to Elastic Search. Default: 1000.
62
+ esConfig = Full path to local elasticsearch.yml. Default: /etc/elasticsearch/elasticsearch.yml
63
+ esPlugins = Full path to local elastic search plugins dir. Default: /usr/local/share/elasticsearch/plugins
64
+
65
+ */
66
+ public class ElasticSearchIndex extends StoreFunc implements StoreFuncInterface {
67
+
68
+ private static final Log LOG = LogFactory.getLog(ElasticSearchIndex.class);
69
+
70
+ protected RecordWriter writer = null;
71
+ protected String idField;
72
+ protected String bulkSize;
73
+ protected String esConfig;
74
+ protected String esPlugins;
75
+
76
+ // For hadoop configuration
77
+ private static final String ES_INDEX_NAME = "elasticsearch.index.name";
78
+ private static final String ES_BULK_SIZE = "elasticsearch.bulk.size";
79
+ private static final String ES_IS_JSON = "elasticsearch.is_json";
80
+ private static final String ES_ID_FIELD_NAME = "elasticsearch.id.field.name";
81
+ private static final String ES_FIELD_NAMES = "elasticsearch.field.names";
82
+ private static final String ES_ID_FIELD = "elasticsearch.id.field";
83
+ private static final String ES_OBJECT_TYPE = "elasticsearch.object.type";
84
+ private static final String PIG_ES_FIELD_NAMES = "elasticsearch.pig.field.names";
85
+
86
+ // Other string constants
87
+ private static final String SLASH = "/";
88
+ private static final String COMMA = ",";
89
+ private static final String LOCAL_SCHEME = "file://";
90
+ private static final String NO_ID_FIELD = "-1";
91
+ private static final String DEFAULT_BULK = "1000";
92
+ private static final String DEFAULT_ES_CONFIG = "/etc/elasticsearch/elasticsearch.yml";
93
+ private static final String DEFAULT_ES_PLUGINS = "/usr/local/share/elasticsearch/plugins";
94
+ private static final String ES_CONFIG_HDFS_PATH = "/tmp/elasticsearch/elasticsearch.yml";
95
+ private static final String ES_PLUGINS_HDFS_PATH = "/tmp/elasticsearch/plugins";
96
+
97
+ public ElasticSearchIndex() {
98
+ this(NO_ID_FIELD, DEFAULT_BULK);
99
+ }
100
+
101
+ public ElasticSearchIndex(String idField, String bulkSize) {
102
+ this(idField, bulkSize, DEFAULT_ES_CONFIG);
103
+ }
104
+
105
+ public ElasticSearchIndex(String idField, String bulkSize, String esConfig) {
106
+ this(idField, bulkSize, esConfig, DEFAULT_ES_PLUGINS);
107
+ }
108
+
109
+ public ElasticSearchIndex(String idField, String bulkSize, String esConfig, String esPlugins) {
110
+ this.idField = idField;
111
+ this.bulkSize = bulkSize;
112
+ this.esConfig = esConfig;
113
+ this.esPlugins = esPlugins;
114
+ }
115
+
116
+ /**
117
+ Check that schema is reasonable and serialize the field names as a string for later use.
118
+ */
119
+ @Override
120
+ public void checkSchema(ResourceSchema s) throws IOException {
121
+ UDFContext context = UDFContext.getUDFContext();
122
+ Properties property = context.getUDFProperties(ResourceSchema.class);
123
+ String fieldNames = "";
124
+ for (String field : s.fieldNames()) {
125
+ fieldNames += field;
126
+ fieldNames += COMMA;
127
+ }
128
+ property.setProperty(PIG_ES_FIELD_NAMES, fieldNames);
129
+ }
130
+
131
+ /**
132
+ Look at passed in location and configuration and set options. Note that, since this
133
+ is called more than once, we need to make sure and not change anything we've already
134
+ set.
135
+ */
136
+ @Override
137
+ public void setStoreLocation(String location, Job job) throws IOException {
138
+ String[] es_store = location.substring(5).split(SLASH);
139
+ if (es_store.length != 2) {
140
+ throw new RuntimeException("Please specify a valid elasticsearch index, eg. es://myindex/myobj");
141
+ }
142
+ Configuration conf = job.getConfiguration();
143
+ // Only set if we haven't already
144
+ if (conf.get(ES_INDEX_NAME) == null) {
145
+ try {
146
+ job.getConfiguration().set(ES_INDEX_NAME, es_store[0]);
147
+ job.getConfiguration().set(ES_OBJECT_TYPE, es_store[1]);
148
+ } catch (ArrayIndexOutOfBoundsException e) {
149
+ throw new RuntimeException("You must specify both an index and an object type.");
150
+ }
151
+ job.getConfiguration().setBoolean(ES_IS_JSON, false);
152
+ job.getConfiguration().set(ES_BULK_SIZE, bulkSize);
153
+ job.getConfiguration().set(ES_ID_FIELD, idField);
154
+
155
+ // Adds the elasticsearch.yml file (esConfig) and the plugins directory (esPlugins) to the distributed cache
156
+ try {
157
+ Path hdfsConfigPath = new Path(ES_CONFIG_HDFS_PATH);
158
+ Path hdfsPluginsPath = new Path(ES_PLUGINS_HDFS_PATH);
159
+
160
+ HadoopUtils.uploadLocalFile(new Path(LOCAL_SCHEME+esConfig), hdfsConfigPath, job.getConfiguration());
161
+ HadoopUtils.shipFileIfNotShipped(hdfsConfigPath, job.getConfiguration());
162
+
163
+ HadoopUtils.uploadLocalFile(new Path(LOCAL_SCHEME+esPlugins), hdfsPluginsPath, job.getConfiguration());
164
+ HadoopUtils.shipArchiveIfNotShipped(hdfsPluginsPath, job.getConfiguration());
165
+
166
+ } catch (Exception e) {
167
+ throw new RuntimeException(e);
168
+ }
169
+
170
+ UDFContext context = UDFContext.getUDFContext();
171
+ Properties property = context.getUDFProperties(ResourceSchema.class);
172
+ job.getConfiguration().set(ES_FIELD_NAMES, property.getProperty(PIG_ES_FIELD_NAMES));
173
+ }
174
+ }
175
+
176
+ @Override
177
+ public OutputFormat getOutputFormat() throws IOException {
178
+ return new ElasticSearchOutputFormat();
179
+ }
180
+
181
+ // Suppressing unchecked warnings for RecordWriter, which is not parameterized by StoreFuncInterface
182
+ @Override
183
+ public void prepareToWrite(@SuppressWarnings("rawtypes") RecordWriter writer) throws IOException {
184
+ this.writer = writer;
185
+ }
186
+
187
+ /**
188
+ Map a tuple object into a map-writable object for elasticsearch.
189
+ */
190
+ @SuppressWarnings("unchecked")
191
+ @Override
192
+ public void putNext(Tuple t) throws IOException {
193
+ UDFContext context = UDFContext.getUDFContext();
194
+ Properties property = context.getUDFProperties(ResourceSchema.class);
195
+ MapWritable record = new MapWritable();
196
+ String[] fieldNames = property.getProperty(PIG_ES_FIELD_NAMES).split(COMMA);
197
+ for (int i = 0; i < t.size(); i++) {
198
+ if (i < fieldNames.length) {
199
+ try {
200
+ record.put(new Text(fieldNames[i]), new Text(t.get(i).toString()));
201
+ } catch (NullPointerException e) {
202
+ //LOG.info("Increment null field counter.");
203
+ }
204
+ }
205
+ }
206
+ try {
207
+ writer.write(NullWritable.get(), record);
208
+ } catch (InterruptedException e) {
209
+ throw new IOException(e);
210
+ }
211
+ }
212
+
213
+ @Override
214
+ public void cleanupOnFailure(String location, Job job) throws IOException {
215
+ }
216
+ }