├── LICENSE.md
├── README.md
├── pom.xml
└── src
├── main
├── java
│ └── com
│ │ └── inin
│ │ └── analytics
│ │ └── elasticsearch
│ │ ├── BaseESMapper.java
│ │ ├── BaseESReducer.java
│ │ ├── ConfigParams.java
│ │ ├── ESEmbededContainer.java
│ │ ├── IndexingPostProcessor.java
│ │ ├── ShardConfig.java
│ │ ├── driver
│ │ └── Driver.java
│ │ ├── example
│ │ ├── ExampleIndexingJob.java
│ │ ├── ExampleIndexingReducerImpl.java
│ │ ├── ExampleJobPrep.java
│ │ └── GenerateData.java
│ │ ├── index
│ │ ├── rotation
│ │ │ ├── ElasticSearchIndexMetadata.java
│ │ │ ├── ElasticsearchIndexRotationManager.java
│ │ │ ├── ElasticsearchIndexRotationManagerNoop.java
│ │ │ ├── ElasticsearchIndexRotationManagerZookeeper.java
│ │ │ ├── ExampleElasticsearchIndexRotationStrategyZookeeper.java
│ │ │ └── RebuildPipelineState.java
│ │ ├── routing
│ │ │ ├── ElasticsearchRoutingStrategy.java
│ │ │ └── ElasticsearchRoutingStrategyV1.java
│ │ └── selector
│ │ │ ├── RealtimeIndexSelectionStrategy.java
│ │ │ └── RealtimeIndexSelectionStrategyLagged.java
│ │ ├── transport
│ │ ├── BaseTransport.java
│ │ ├── HDFSSnapshotTransport.java
│ │ ├── LocalFSSnapshotTransport.java
│ │ ├── S3SnapshotTransport.java
│ │ └── SnapshotTransportStrategy.java
│ │ └── util
│ │ ├── DateTimeDeserializer.java
│ │ ├── DateTimeSerializer.java
│ │ ├── GsonFactory.java
│ │ ├── LocalDateDeserializer.java
│ │ ├── LocalDateSerializer.java
│ │ └── MurmurHash.java
└── resources
│ ├── hadoop-job.xml
│ └── logback.xml
└── test
└── java
└── com
└── inin
└── analytics
├── DateSerializerTest.java
├── ElasticsearchRoutingStrategyV1Test.java
└── IndexRotationStrategyZookeeperTest.java
/LICENSE.md:
--------------------------------------------------------------------------------
1 | MIT License
2 |
3 | Copyright (c) 2024 Genesys Cloud Services, Inc.
4 |
5 | Permission is hereby granted, free of charge, to any person obtaining a copy
6 | of this software and associated documentation files (the "Software"), to deal
7 | in the Software without restriction, including without limitation the rights
8 | to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
9 | copies of the Software, and to permit persons to whom the Software is
10 | furnished to do so, subject to the following conditions:
11 |
12 | The above copyright notice and this permission notice shall be included in all
13 | copies or substantial portions of the Software.
14 |
15 | THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
16 | IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
17 | FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
18 | AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
19 | LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
20 | OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
21 | SOFTWARE.
--------------------------------------------------------------------------------
/README.md:
--------------------------------------------------------------------------------
1 | # Elasticsearch-Lambda
2 |
3 | ## What is Lambda
4 |
5 | For a primer on Lambda Architecture see [http://lambda-architecture.net/]. At Interactive Intelligence we're applying Lamda architecture to Elasticsearch. In our case that means we're streaming data into Elasticsearch in real time generated by Storm, and then re-building the whole thing every night in Hadoop. This allows some architecture nicities such as
6 |
7 | * Changes to analyzers & tokenizers are rolled out for all historical data every night, automatically
8 | * New features & bug fixes that affect the data being indexed are rolled out every night, automatically. No data repair/backpopulation scripts are ever required.
9 | * Tune the # shards and shard routing strategies on data already written. Changes roll out every night, automatically.
10 | * With the button that rebuilds the cluster getting hit nightly, it is a well oiled button.
11 | * If data gets corrupt, no heroics are required. Hit the rebuild button and grab a beer.
12 | * Backups? Why bother when you can just hit the rebuild button?
13 |
14 | Obviously there's a decent bit of work up front to get all this working. Being a fairly generic problem, we decided to open source our infastructure.
15 |
16 | ## Features
17 |
18 | A new way to bulk load elasticsearch from hadoop
19 |
20 | * Build indexes offline, without touching your production cluster
21 | * Run Elasticsearch unmodified, entirely within YARN
22 | * Build snapshots of indexes without requiring enough disk space on task trackers to hold an entire index
23 | * Load those indexes into your cluster using the snapshot restore functionality built into Elasticsearch
24 | * .. and more to come. We're in the process of pulling as much into this repo as we can.
25 |
26 |
27 | ## How it works
28 |
29 | The meat is in BaseEsReducer where individual reducer tasks recieve all the data for a single shard of a single index. It creates an embeded Elasticsearch instance, bulk loads it locally in-jvm, and then creates a snapshot. Discovery is disabled and the elasticsearch instances do not form a cluster with each other. Once bulk loading a shard is complete it is flushed, optimized, snapshotted, and then transfered to a snapshot repository (S3, HDFS, or Local FS). After the job is complete, any shards that have no data get placeholder shards generated to make the index complete.
30 |
31 | By making reducers only responsible for a single shard worth of data at a time, the total disk space required on task trackers is roughly
32 |
33 | (shard data) + (shard snapshot) * (num reducers per task tracker)
34 |
35 | After indexes have been generated they can be loaded in using the snapshot restore functionality built into Elasticsearch. The index promotion process maintains state in Zookeeper. This is in the process of being open sourced.
36 |
37 | ## Maven
38 | ```XML
39 |
40 | oss-sonatype
41 | oss-sonatype
42 | https://oss.sonatype.org/content/groups/public/
43 |
44 |
45 |
46 | elasticsearch-lambda
47 | com.inin.analytics
48 | 1.0.25
49 |
50 | ```
51 |
52 | ## Shard routing
53 |
54 | In order to index 1 shard per reducer at a time, elasticsearch-lambda relies on manual shard routing. If you've got big indexes (probably why you're here), then you'll almost certainly want a custom routing strategy so that searches can hit a subset of shards.
55 |
56 | To create your own you would implement the ElasticsearchRoutingStrategy interface and make use of it during the setup method of the ExampleJobPrep job. The default works as such:
57 |
58 | ElasticsearchRoutingStrategyV1: Two parameters: numShards & numShardsPerOrg. A nieve apprach would be routing all data for 1 customer to 1 shard. To avoid hotspotting shards with large customers, this lets you spread the load across multiple shards. For example with 10 shards and 3 per customer, customer A might sit on shards 1,3,5 while customer B sits on shards 2,3,8. Setting the inputs to 10 & 10 would spread all customers evenly across all 10 shards.
59 |
60 | ## EMR Example Steps
61 |
62 | * generateExampleData 1000 hdfs:///tmp/test/data
63 | * examplePrep hdfs:///tmp/test/data/ hdfs:///tmp/test/json/ _rebuild_20141030012508 5 2
64 | * esIndexRebuildExample hdfs:///tmp/test/json/ /media/ephemeral0/tmp/bulkload/ hdfs:///tmp/snapshotrepo/ my_backup /media/ephemeral0/tmp/esrawdata/ 1 5 100 hdfs:///tmp/manifest/
65 |
66 |
67 | ## Running Configs (for eclipse/IDE)
68 | You can experiment via these run configs ran in series
69 |
70 | * com.inin.analytics.elasticsearch.driver.Driver
71 |
72 | Lets build some dummy data
73 |
74 | * generateExampleData 1000 file:///tmp/data/part2
75 |
76 | Prepare some data for the indexing job
77 |
78 | * examplePrep /tmp/data/part2 /tmp/datajson/ _rebuild_20141030012508 5 2
79 |
80 | Build Elasticsearch indexes, snapshot them, and transport them to a snapshot repository on hdfs (s3 paths also allowed)
81 |
82 | * esIndexRebuildExample /tmp/datajson/ /tmp/bulkload110/ hdfs:///tmp/snapshotrepo110/ my_backup /tmp/esrawdata1010/ 1 5 2 /tmp/manifest110/
83 |
84 | ### Can I use HDFS or NFS for Elasticsearch data?
85 |
86 | Elasticsearch does not currently support backing it's data with HDFS, so this project makes use of local disks on the task trackers. Given that Solr Cloud already supports HDFS backed data, it's concievable that one day Elasticsearch might.
87 |
88 | When considering NFS you must first consider how different hadoop distributions have implemented it. The apache version of hadoop implements NFS with large local disk buffers, so it may or may not save you any trouble. The Mapr NFS implementation is more native and performant. In our tests, running Elasticsearch on YARN and backing the data directories by NFS mounts backed by MapR-FS ran roughly half as fast. While impressive, it's up to you to balance the cost of using MapR for it's NFS cabilitiy to run Elasticsearch. Note, this requires substituting Elasticsearch's locking mechanism for a non-filesystem based implementation.
89 |
90 |
--------------------------------------------------------------------------------
/pom.xml:
--------------------------------------------------------------------------------
1 |
3 | 4.0.0
4 |
5 | com.inin.analytics
6 | elasticsearch-lambda
7 | jar
8 | 1.2.1
9 | elasticsearch-lambda
10 | Framework For Lambda Architecture on Elasticsearch
11 | https://github.com/drewdahlke/elasticsearch-lambda
12 |
13 | UTF-8
14 | 1.7
15 | 4.11
16 | 1.9.5
17 | 0.1.93-SNAPSHOT
18 | 2.2.0
19 | 2.4
20 | 2.5
21 | 1.4
22 | 2.4.0
23 | /tmp
24 | 2.7.2
25 | 1.0.0
26 | 2.3
27 | 1.9.2
28 | 1.0.13
29 | 1.9.38
30 | 1.7.5
31 | 1.7.5
32 |
33 |
34 |
35 |
36 | The Apache License, Version 2.0
37 | http://www.apache.org/licenses/LICENSE-2.0.txt
38 |
39 |
40 |
41 |
42 | https://github.com/drewdahlke/elasticsearch-lambda.git
43 | https://github.com/drewdahlke/elasticsearch-lambda.git
44 | https://github.com/drewdahlke/elasticsearch-lambda.git
45 |
46 |
47 |
48 |
49 |
50 | repository.springframework.maven.release
51 | Spring Framework Maven Release Repository
52 | http://maven.springframework.org/release
53 |
54 | false
55 |
56 |
57 |
58 | conjars.org
59 | http://conjars.org/repo
60 |
61 |
62 | Codehaus Snapshots
63 | http://repository.codehaus.org/
64 |
65 |
66 | central
67 | Maven Repository Switchboard
68 | default
69 | http://repo1.maven.org/maven2
70 |
71 | false
72 |
73 |
74 |
75 | oss-sonatype
76 | oss-sonatype
77 | https://oss.sonatype.org/content/groups/public
78 |
79 |
80 |
81 |
82 |
83 |
84 | commons-codec
85 | commons-codec
86 | ${commons.codec.version}
87 |
88 |
89 | org.codehaus.jackson
90 | jackson-mapper-lgpl
91 | ${jackson.version}
92 |
93 |
94 | org.codehaus.jackson
95 | jackson-core-lgpl
96 | ${jackson.version}
97 |
98 |
99 | jackson-mapper-asl
100 | org.codehaus.jackson
101 | ${jackson.version}
102 |
103 |
104 | org.codehaus.jackson
105 | jackson-core-asl
106 | ${jackson.version}
107 |
108 |
109 | org.apache.avro
110 | avro
111 | ${org.apache.avro.version}
112 | provided
113 |
114 |
115 |
116 | us.monoid.web
117 | resty
118 | 0.3.2
119 |
120 |
121 | joda-time
122 | joda-time
123 | ${joda.version}
124 |
125 |
126 |
127 |
128 |
129 | org.apache.curator
130 | curator-recipes
131 | ${curator.version}
132 |
133 |
134 | org.apache.commons
135 | commons-math
136 |
137 |
138 | org.javassist
139 | javassist
140 |
141 |
142 | slf4j-log4j12
143 | org.slf4j
144 |
145 |
146 | log4j
147 | log4j
148 |
149 |
150 |
151 |
152 |
153 | org.apache.curator
154 | curator-framework
155 | 2.3.0
156 |
157 |
158 | slf4j-api
159 | org.slf4j
160 |
161 |
162 |
163 |
164 |
165 | com.amazonaws
166 | aws-java-sdk
167 | ${aws-java-sdk.version}
168 |
169 |
170 | com.fasterxml.jackson.core
171 | jackson-core
172 |
173 |
174 | com.fasterxml.jackson.core
175 | jackson-annotations
176 |
177 |
179 |
180 | commons-codec
181 | commons-codec
182 |
183 |
184 | commons-logging
185 | commons-logging
186 |
187 |
188 |
189 |
190 |
191 |
192 | org.apache.mrunit
193 | mrunit
194 | ${mrunit.version}
195 | hadoop2
196 | provided
197 |
198 |
199 | org.elasticsearch
200 | elasticsearch
201 | 1.6.0
202 |
203 |
204 |
205 | com.google.code.gson
206 | gson
207 | 2.3.1
208 | provided
209 |
210 |
211 |
212 |
213 | org.slf4j
214 | slf4j-api
215 | ${slf4j.version}
216 |
217 |
218 | org.slf4j
219 | jcl-over-slf4j
220 | ${slf4j.version}
221 |
222 |
223 | org.slf4j
224 | log4j-over-slf4j
225 | ${slf4j.version}
226 |
227 |
228 | org.slf4j
229 | jul-to-slf4j
230 | ${slf4j.version}
231 |
232 |
233 | ch.qos.logback
234 | logback-classic
235 | ${logback.version}
236 |
237 |
238 |
239 | ch.qos.logback
240 | logback-core
241 | ${logback.version}
242 |
243 |
244 |
245 |
246 | org.slf4j
247 | slf4j-api
248 |
249 |
250 |
251 |
252 | commons-lang
253 | commons-lang
254 | ${apache.commons-lang.version}
255 | provided
256 |
257 |
258 | commons-io
259 | commons-io
260 | ${apache.commons-io.version}
261 | provided
262 |
263 |
264 | org.apache.commons
265 | commons-compress
266 | 1.5
267 | provided
268 |
269 |
270 | commons-httpclient
271 | commons-httpclient
272 | 3.1
273 | provided
274 |
275 |
276 | commons-codec
277 | commons-codec
278 |
279 |
280 |
281 |
282 |
283 | org.slf4j
284 | slf4j-api
285 | 1.7.5
286 |
287 |
288 |
289 |
290 | org.slf4j
291 | jul-to-slf4j
292 | ${slf4j.version}
293 |
294 |
295 | org.slf4j
296 | log4j-over-slf4j
297 | ${slf4j.version}
298 |
299 |
300 |
301 | org.apache.curator
302 | curator-test
303 | ${curator.version}
304 |
305 |
306 | org.apache.commons
307 | commons-math
308 |
309 |
310 | org.javassist
311 | javassist
312 |
313 |
314 | provided
315 |
316 |
317 | org.javassist
318 | javassist
319 | 3.18.1-GA
320 |
321 |
322 |
323 |
324 | junit
325 | junit
326 | ${junit.version}
327 | test
328 |
329 |
330 |
331 | org.mockito
332 | mockito-all
333 | ${mockito.version}
334 | test
335 |
336 |
337 |
338 | org.apache.hadoop
339 | hadoop-mapreduce-client-app
340 | ${hadoop.core.version}
341 |
342 |
343 | jackson-core-asl
344 | org.codehaus.jackson
345 |
346 |
347 | jackson-mapper-asl
348 | org.codehaus.jackson
349 |
350 |
351 | avro
352 | org.apache.avro
353 |
354 |
355 | slf4j-log4j12
356 | org.slf4j
357 |
358 |
359 | log4j
360 | log4j
361 |
362 |
363 | provided
364 |
365 |
366 | org.apache.hadoop
367 | hadoop-common
368 | ${hadoop.core.version}
369 |
370 |
371 | commons-codec
372 | commons-codec
373 |
374 |
375 | jackson-core-asl
376 | org.codehaus.jackson
377 |
378 |
379 | jackson-mapper-asl
380 | org.codehaus.jackson
381 |
382 |
383 | avro
384 | org.apache.avro
385 |
386 |
387 | slf4j-log4j12
388 | org.slf4j
389 |
390 |
391 | log4j
392 | log4j
393 |
394 |
395 | provided
396 |
397 |
398 | org.apache.hadoop
399 | hadoop-hdfs
400 | ${hadoop.core.version}
401 |
402 |
403 | commons-codec
404 | commons-codec
405 |
406 |
407 | jackson-core-asl
408 | org.codehaus.jackson
409 |
410 |
411 | jackson-mapper-asl
412 | org.codehaus.jackson
413 |
414 |
415 | log4j
416 | log4j
417 |
418 |
419 | provided
420 |
421 |
422 |
423 |
424 |
425 | Drew Dahlke
426 | justin.dahlke@gmail.com
427 | Interactive Intelligence
428 | http://www.inin.com
429 |
430 |
431 | Michael Mulligan
432 | Michael.Mulligan@inin.com
433 | Interactive Intelligence
434 | http://www.inin.com
435 |
436 |
437 |
438 |
439 |
440 | ossrh
441 | https://oss.sonatype.org/content/repositories/snapshots
442 |
443 |
444 | ossrh
445 | https://oss.sonatype.org/service/local/staging/deploy/maven2/
446 |
447 |
448 |
449 |
450 |
451 |
452 |
453 | org.sonatype.plugins
454 | nexus-staging-maven-plugin
455 | 1.6.3
456 | true
457 |
458 | ossrh
459 | https://oss.sonatype.org/
460 | true
461 |
462 |
463 |
464 | org.apache.maven.plugins
465 | maven-source-plugin
466 | 2.2.1
467 |
468 |
469 | attach-sources
470 |
471 | jar-no-fork
472 |
473 |
474 |
475 |
476 |
477 | org.apache.maven.plugins
478 | maven-gpg-plugin
479 | 1.5
480 |
481 | gpg
482 |
483 |
484 |
485 | sign-artifacts
486 | verify
487 |
488 | sign
489 |
490 |
491 |
492 |
493 |
494 | org.apache.maven.plugins
495 | maven-javadoc-plugin
496 |
497 |
498 | -Xdoclint:none
499 |
500 |
501 |
502 | org.apache.maven.plugins
503 | maven-javadoc-plugin
504 | 2.9.1
505 |
506 |
507 | attach-javadocs
508 |
509 | jar
510 |
511 |
512 |
513 |
514 |
515 | org.apache.maven.plugins
516 | maven-surefire-plugin
517 | ${surefire.version}
518 |
519 | ${java.io.tmpdir}
520 |
521 |
522 |
523 | org.apache.maven.plugins
524 | maven-compiler-plugin
525 | 3.1
526 |
527 | ${java-version}
528 | ${java-version}
529 |
530 |
531 |
532 | maven-assembly-plugin
533 | 2.2.1
534 |
535 |
536 | src/main/resources/hadoop-job.xml
537 |
538 |
539 |
540 | com.inin.analytics.elasticsearch.driver.Driver
541 |
542 |
543 |
544 |
545 |
546 | make-assembly
547 | package
548 |
549 | single
550 |
551 |
552 |
553 |
554 |
555 | org.codehaus.mojo
556 | exec-maven-plugin
557 | 1.2.1
558 |
559 |
560 |
561 | java
562 |
563 |
564 |
565 |
566 | com.inin.analytics.elasticlearch.driver.Driver
567 |
568 |
569 |
570 |
571 |
572 |
--------------------------------------------------------------------------------
/src/main/java/com/inin/analytics/elasticsearch/BaseESMapper.java:
--------------------------------------------------------------------------------
1 | package com.inin.analytics.elasticsearch;
2 |
3 | import java.io.IOException;
4 |
5 | import org.apache.hadoop.io.Text;
6 | import org.apache.hadoop.mapred.JobConf;
7 | import org.apache.hadoop.mapred.Mapper;
8 | import org.apache.hadoop.mapred.OutputCollector;
9 | import org.apache.hadoop.mapred.Reporter;
10 |
11 | /**
12 | * This class acts as a passthrough routing data to the reducer. Your input should be keyed off of
13 | * [index name] | [index type]
14 | *
15 | * The values should be the raw json payloads to send to ES.
16 | * @author drew
17 | *
18 | */
19 | public class BaseESMapper implements Mapper {
20 | public void configure(JobConf job) {
21 | }
22 |
23 | public void map(Text indexAndRouting, Text documentPayload, OutputCollector output, Reporter reporter) throws IOException {
24 | /**
25 | * Reducer key looks like this [indexName]|[routing hash] value [doc type]|[doc id]|json
26 | *
27 | */
28 | output.collect(indexAndRouting, documentPayload);
29 | }
30 |
31 | public void close() throws IOException {
32 | }
33 |
34 | }
--------------------------------------------------------------------------------
/src/main/java/com/inin/analytics/elasticsearch/BaseESReducer.java:
--------------------------------------------------------------------------------
1 | package com.inin.analytics.elasticsearch;
2 |
3 | import static org.elasticsearch.common.settings.ImmutableSettings.settingsBuilder;
4 |
5 | import java.io.File;
6 | import java.io.IOException;
7 | import java.util.Arrays;
8 | import java.util.Iterator;
9 |
10 | import org.apache.commons.io.FileUtils;
11 | import org.apache.commons.lang.StringUtils;
12 | import org.apache.hadoop.io.NullWritable;
13 | import org.apache.hadoop.io.Text;
14 | import org.apache.hadoop.mapred.JobConf;
15 | import org.apache.hadoop.mapred.OutputCollector;
16 | import org.apache.hadoop.mapred.Reducer;
17 | import org.apache.hadoop.mapred.Reporter;
18 | import org.elasticsearch.action.ActionFuture;
19 | import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequest;
20 | import org.elasticsearch.action.admin.indices.delete.DeleteIndexResponse;
21 | import org.elasticsearch.action.index.IndexResponse;
22 | import org.elasticsearch.cluster.metadata.IndexMetaData;
23 | import com.inin.analytics.elasticsearch.transport.SnapshotTransportStrategy;
24 |
25 | public abstract class BaseESReducer implements Reducer {
26 | public static final char TUPLE_SEPARATOR = '|';
27 | public static final char DIR_SEPARATOR = '/';
28 |
29 | public static enum JOB_COUNTER {
30 | TIME_SPENT_INDEXING_MS, TIME_SPENT_FLUSHING_MS, TIME_SPENT_MERGING_MS, TIME_SPENT_SNAPSHOTTING_MS, TIME_SPENT_TRANSPORTING_SNAPSHOT_MS, INDEXING_DOC_FAIL, INDEX_DOC_CREATED, INDEX_DOC_NOT_CREATED
31 | }
32 |
33 | // We prefix all snapshots with the word snapshot
34 | public static final String SNAPSHOT_NAME = "snapshot";
35 |
36 | // The local filesystem location that ES will write the snapshot out to
37 | private String snapshotWorkingLocation;
38 |
39 | // Where the snapshot will be moved to. Typical use case would be to throw it onto S3
40 | private String snapshotFinalDestination;
41 |
42 | // The name of a snapshot repo. We'll enumerate that on each job run so that the repo names are unique across rebuilds
43 | private String snapshotRepoName;
44 |
45 | // Local filesystem location where index data is built
46 | private String esWorkingDir;
47 |
48 | // The partition of data this reducer is serving. Useful for making directories unique if running multiple reducers on a task tracker
49 | private String partition;
50 |
51 | // The container handles spinning up our embedded elasticsearch instance
52 | private ESEmbededContainer esEmbededContainer;
53 |
54 | private ShardConfig shardConfig;
55 |
56 | // Hold onto some frequently generated objects to cut down on GC overhead
57 | private String indexType;
58 | private String docId;
59 | private String pre;
60 | private String json;
61 |
62 | @Override
63 | public void configure(JobConf job) {
64 | partition = job.get("mapred.task.partition");
65 | String attemptId = job.get("mapred.task.id");
66 |
67 | // If running multiple reducers on a node, the node needs a unique name & data directory hence the random number we append
68 | snapshotWorkingLocation = job.get(ConfigParams.SNAPSHOT_WORKING_LOCATION_CONFIG_KEY.toString()) + partition + attemptId + DIR_SEPARATOR;
69 | snapshotFinalDestination = job.get(ConfigParams.SNAPSHOT_FINAL_DESTINATION.toString());
70 | snapshotRepoName = job.get(ConfigParams.SNAPSHOT_REPO_NAME_CONFIG_KEY.toString());
71 | esWorkingDir = job.get(ConfigParams.ES_WORKING_DIR.toString()) + partition + attemptId + DIR_SEPARATOR;
72 | if(shardConfig == null) {
73 | shardConfig = getShardConfig(job);
74 | }
75 | }
76 |
77 | public void setShardConfig(ShardConfig shardConfig) {
78 | this.shardConfig = shardConfig;
79 | }
80 |
81 | private void init(String index) {
82 | String templateName = getTemplateName();
83 | String templateJson = getTemplate();
84 |
85 | ESEmbededContainer.Builder builder = new ESEmbededContainer.Builder()
86 | .withNodeName("embededESTempLoaderNode" + partition)
87 | .withWorkingDir(esWorkingDir)
88 | .withClusterName("bulkLoadPartition:" + partition)
89 | .withSnapshotWorkingLocation(snapshotWorkingLocation)
90 | .withSnapshotRepoName(snapshotRepoName);
91 |
92 | if(templateName != null && templateJson != null) {
93 | builder.withTemplate(templateName, templateJson);
94 | }
95 |
96 | if(esEmbededContainer == null) {
97 | esEmbededContainer = builder.build();
98 | }
99 |
100 | // Create index
101 | esEmbededContainer.getNode().client().admin().indices().prepareCreate(index).setSettings(settingsBuilder()
102 | .put("index.number_of_replicas", 0)
103 | .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, shardConfig.getShardsForIndex(index))
104 | ).get();
105 | }
106 |
107 | /**
108 | * Provide the JSON contents of the index template. This is your hook for configuring ElasticSearch.
109 | *
110 | * http://www.elasticsearch.org/guide/en/elasticsearch/reference/current/indices-templates.html
111 | *
112 | * @return String
113 | */
114 | public abstract String getTemplate();
115 |
116 |
117 | /**
118 | * Provide a ShardConfig which provides the number of shards per index and the number of
119 | * shards to split organizations across. The number can be uniform across indices or a mapping
120 | * can be provided to enable per-index configuration values.
121 | *
122 | * @param job
123 | * @return ShardConfig
124 | */
125 | public abstract ShardConfig getShardConfig(JobConf job);
126 |
127 | /**
128 | * Provide an all lower case template name
129 | *
130 | * @return String
131 | */
132 | public abstract String getTemplateName();
133 |
134 | @Override
135 | public void reduce(Text docMetaData, Iterator documentPayloads, OutputCollector output, final Reporter reporter) throws IOException {
136 | String[] pieces = StringUtils.split(docMetaData.toString(), TUPLE_SEPARATOR);
137 | String indexName = pieces[0];
138 | String routing = pieces[1];
139 | init(indexName);
140 |
141 | long start = System.currentTimeMillis();
142 | while(documentPayloads.hasNext()) {
143 | Text line = documentPayloads.next();
144 | if(line == null) {
145 | continue;
146 | }
147 |
148 | pieces = StringUtils.split(line.toString(), TUPLE_SEPARATOR);
149 | indexType = pieces[0];
150 | docId = pieces[1];
151 | pre = indexType + TUPLE_SEPARATOR + docId + TUPLE_SEPARATOR;
152 | json = line.toString().substring(pre.length());
153 |
154 | IndexResponse response = esEmbededContainer.getNode().client().prepareIndex(indexName, indexType).setId(docId).setRouting(routing).setSource(json).execute().actionGet();
155 | if(response.isCreated()) {
156 | reporter.incrCounter(JOB_COUNTER.INDEX_DOC_CREATED, 1l);
157 | } else {
158 | reporter.incrCounter(JOB_COUNTER.INDEX_DOC_NOT_CREATED, 1l);
159 | }
160 | }
161 |
162 | reporter.incrCounter(JOB_COUNTER.TIME_SPENT_INDEXING_MS, System.currentTimeMillis() - start);
163 |
164 | snapshot(indexName, reporter);
165 | output.collect(NullWritable.get(), new Text(indexName));
166 | }
167 |
168 | @Override
169 | public void close() throws IOException {
170 | if(esEmbededContainer != null) {
171 | esEmbededContainer.getNode().close();
172 | while(!esEmbededContainer.getNode().isClosed());
173 | FileUtils.deleteDirectory(new File(snapshotWorkingLocation));
174 | }
175 | }
176 |
177 | public void snapshot(String index, Reporter reporter) throws IOException {
178 | esEmbededContainer.snapshot(Arrays.asList(index), SNAPSHOT_NAME, snapshotRepoName, reporter);
179 |
180 | // Delete the index to free up that space
181 | ActionFuture response = esEmbededContainer.getNode().client().admin().indices().delete(new DeleteIndexRequest(index));
182 | while(!response.isDone());
183 |
184 | // Move the shard snapshot to the destination
185 | long start = System.currentTimeMillis();
186 | SnapshotTransportStrategy.get(snapshotWorkingLocation, snapshotFinalDestination).execute(SNAPSHOT_NAME, index);
187 | reporter.incrCounter(JOB_COUNTER.TIME_SPENT_TRANSPORTING_SNAPSHOT_MS, System.currentTimeMillis() - start);
188 |
189 | esEmbededContainer.deleteSnapshot(SNAPSHOT_NAME, snapshotRepoName);
190 | }
191 | }
--------------------------------------------------------------------------------
/src/main/java/com/inin/analytics/elasticsearch/ConfigParams.java:
--------------------------------------------------------------------------------
1 | package com.inin.analytics.elasticsearch;
2 |
3 | public enum ConfigParams {
4 | SNAPSHOT_WORKING_LOCATION_CONFIG_KEY,
5 | SNAPSHOT_REPO_NAME_CONFIG_KEY,
6 | SNAPSHOT_FINAL_DESTINATION,
7 | ES_WORKING_DIR,
8 | NUM_SHARDS_PER_INDEX,
9 | NUM_SHARDS_PER_ORGANIZATION
10 | }
11 |
--------------------------------------------------------------------------------
/src/main/java/com/inin/analytics/elasticsearch/ESEmbededContainer.java:
--------------------------------------------------------------------------------
1 | package com.inin.analytics.elasticsearch;
2 |
3 | import static org.elasticsearch.node.NodeBuilder.nodeBuilder;
4 |
5 | import java.util.HashMap;
6 | import java.util.List;
7 | import java.util.Map;
8 |
9 | import org.apache.hadoop.mapred.Reporter;
10 | import org.elasticsearch.action.admin.cluster.snapshots.get.GetSnapshotsRequest;
11 | import org.elasticsearch.action.admin.cluster.snapshots.get.GetSnapshotsResponse;
12 | import org.elasticsearch.cluster.metadata.IndexMetaData;
13 | import org.elasticsearch.common.settings.ImmutableSettings;
14 | import org.elasticsearch.common.settings.Settings;
15 | import org.elasticsearch.common.unit.TimeValue;
16 | import org.elasticsearch.node.Node;
17 | import org.elasticsearch.plugins.PluginsService;
18 | import org.elasticsearch.snapshots.SnapshotInfo;
19 | import org.slf4j.Logger;
20 | import org.slf4j.LoggerFactory;
21 |
22 | import com.google.common.base.Preconditions;
23 |
24 | /**
25 | * Builds an embedded elasticsearch instance and configures it for you
26 | *
27 | * @author drew
28 | *
29 | */
30 | public class ESEmbededContainer {
31 | private Node node;
32 | private long DEFAULT_TIMEOUT_MS = 60 * 30 * 1000;
33 | private static Integer MAX_MERGED_SEGMENT_SIZE_MB = 256;
34 | private static transient Logger logger = LoggerFactory.getLogger(ESEmbededContainer.class);
35 |
36 | public void snapshot(List index, String snapshotName, String snapshotRepoName, Reporter reporter) {
37 | snapshot(index, snapshotName, snapshotRepoName, DEFAULT_TIMEOUT_MS, reporter);
38 | }
39 |
40 | /**
41 | * Flush, optimize, and snapshot an index. Block until complete.
42 | *
43 | * @param indicies
44 | * @param snapshotName
45 | * @param snapshotRepoName
46 | * @param timeoutMS
47 | * @param reporter
48 | */
49 | public void snapshot(List indicies, String snapshotName, String snapshotRepoName, long timeoutMS, Reporter reporter) {
50 | /* Flush & optimize before the snapshot.
51 | *
52 | * TODO: Long operations could block longer that the container allows an operation to go
53 | * unresponsive b/f killing. We need to issue the request and poll the future waiting on the
54 | * operation to succeed, but update a counter or something to let the hadoop framework
55 | * know the process is still alive.
56 | */
57 | TimeValue v = new TimeValue(timeoutMS);
58 | for(String index : indicies) {
59 | long start = System.currentTimeMillis();
60 |
61 | // Flush
62 | node.client().admin().indices().prepareFlush(index).get(v);
63 | if(reporter != null) {
64 | reporter.incrCounter(BaseESReducer.JOB_COUNTER.TIME_SPENT_FLUSHING_MS, System.currentTimeMillis() - start);
65 | }
66 |
67 | // Merge
68 | start = System.currentTimeMillis();
69 | node.client().admin().indices().prepareOptimize(index).get(v);
70 | if(reporter != null) {
71 | reporter.incrCounter(BaseESReducer.JOB_COUNTER.TIME_SPENT_MERGING_MS, System.currentTimeMillis() - start);
72 | }
73 | }
74 |
75 | // Snapshot
76 | long start = System.currentTimeMillis();
77 | node.client().admin().cluster().prepareCreateSnapshot(snapshotRepoName, snapshotName).setIndices((String[]) indicies.toArray(new String[0])).execute();
78 |
79 | // ES snapshot restore ignores timers and will block no more than 30s :( You have to block & poll to make sure it's done
80 | blockForSnapshot(snapshotRepoName, indicies, timeoutMS);
81 |
82 | if(reporter != null) {
83 | reporter.incrCounter(BaseESReducer.JOB_COUNTER.TIME_SPENT_SNAPSHOTTING_MS, System.currentTimeMillis() - start);
84 | }
85 |
86 | }
87 |
88 | /**
89 | * Block for index snapshots to be complete
90 | *
91 | * @param snapshotRepoName
92 | * @param indicies
93 | * @param timeoutMS
94 | */
95 | private void blockForSnapshot(String snapshotRepoName, List indicies, long timeoutMS) {
96 | long start = System.currentTimeMillis();
97 | while(System.currentTimeMillis() - start < timeoutMS) {
98 |
99 | GetSnapshotsResponse repos = node.client().admin().cluster().getSnapshots(new GetSnapshotsRequest(snapshotRepoName)).actionGet();
100 | for(SnapshotInfo i : repos.getSnapshots()) {
101 | if(i.state().completed() && i.successfulShards() == i.totalShards() && i.totalShards() >= indicies.size()) {
102 | logger.info("Snapshot completed {} out of {} indicies. Snapshot state {}. ", i.successfulShards(), i.totalShards(), i.state().completed());
103 | return;
104 | } else {
105 | logger.info("Snapshotted {} out of {} indicies, polling for completion. Snapshot state {}.", i.successfulShards(), i.totalShards(), i.state().completed());
106 | }
107 | }
108 | try {
109 | // Don't slam ES with snapshot status requests in a tight loop
110 | Thread.sleep(1000);
111 | } catch (InterruptedException e) {
112 | Thread.currentThread().interrupt();
113 | }
114 | }
115 | }
116 |
117 | public void deleteSnapshot(String snapshotName, String snapshotRepoName) {
118 | node.client().admin().cluster().prepareDeleteSnapshot(snapshotRepoName, snapshotName).execute().actionGet();
119 | }
120 |
121 | public static class Builder {
122 | private ESEmbededContainer container;
123 | private String nodeName;
124 | private String workingDir;
125 | private String clusterName;
126 | private String templateName;
127 | private String templateSource;
128 | private String snapshotWorkingLocation;
129 | private String snapshotRepoName;
130 | private boolean memoryBackedIndex = false;
131 |
132 | public ESEmbededContainer build() {
133 | Preconditions.checkNotNull(nodeName);
134 | Preconditions.checkNotNull(workingDir);
135 | Preconditions.checkNotNull(clusterName);
136 |
137 | org.elasticsearch.common.settings.ImmutableSettings.Builder builder = ImmutableSettings.builder()
138 | .put("http.enabled", false) // Disable HTTP transport, we'll communicate inner-jvm
139 | .put("processors", 1) // We could experiment ramping this up to match # cores - num reducers per node
140 | .put("node.name", nodeName)
141 | .put("path.data", workingDir)
142 | .put("plugins." + PluginsService.LOAD_PLUGIN_FROM_CLASSPATH, true) // Allow plugins if they're bundled in with the uuberjar
143 | .put("index.refresh_interval", -1)
144 | .put("index.translog.flush_threshold_size", "128mb") // Aggressive flushing helps keep the memory footprint below the yarn container max. TODO: Make configurable
145 | .put("bootstrap.mlockall", true)
146 | .put("cluster.routing.allocation.disk.watermark.low", 99) // Nodes don't form a cluster, so routing allocations don't matter
147 | .put("cluster.routing.allocation.disk.watermark.high", 99)
148 | .put("index.load_fixed_bitset_filters_eagerly", false)
149 | .put("indices.store.throttle.type", "none") // Allow indexing to max out disk IO
150 | .put("indices.memory.index_buffer_size", "5%") // The default 10% is a bit large b/c it's calculated against JVM heap size & not Yarn container allocation. Choosing a good value here could be made smarter.
151 | .put("index.merge.policy.max_merged_segment", MAX_MERGED_SEGMENT_SIZE_MB + "mb") // The default 5gb segment max size is too large for the typical hadoop node
152 | //.put("index.merge.policy.max_merge_at_once", 10)
153 | .put("index.merge.policy.segments_per_tier", 4)
154 | .put("index.merge.scheduler.max_thread_count", 1)
155 | .put("path.repo", snapshotWorkingLocation)
156 | .put("index.compound_format", false) // Explicitly disable compound files
157 | //.put("index.codec", "best_compression") // Lucene 5/ES 2.0 feature to play with when that's out
158 | .put("indices.fielddata.cache.size", "0%");
159 |
160 | if(memoryBackedIndex) {
161 | builder.put("index.store.type", "memory");
162 | }
163 | Settings nodeSettings = builder.build();
164 |
165 | // Create the node
166 | container.setNode(nodeBuilder()
167 | .client(false) // It's a client + data node
168 | .local(true) // Tell ES cluster discovery to be inner-jvm only, disable HTTP based node discovery
169 | .clusterName(clusterName)
170 | .settings(nodeSettings)
171 | .build());
172 |
173 | // Start ES
174 | container.getNode().start();
175 |
176 | // Configure the cluster with an index template mapping
177 | if(templateName != null && templateSource != null) {
178 | container.getNode().client().admin().indices().preparePutTemplate(templateName).setSource(templateSource).get();
179 | }
180 |
181 | // Create the snapshot repo
182 | if(snapshotWorkingLocation != null && snapshotRepoName != null) {
183 | Map settings = new HashMap<>();
184 | settings.put("location", snapshotWorkingLocation);
185 | settings.put("compress", true);
186 | settings.put("max_snapshot_bytes_per_sec", "400mb"); // The default 20mb/sec is very slow for a local disk to disk snapshot
187 | container.getNode().client().admin().cluster().preparePutRepository(snapshotRepoName).setType("fs").setSettings(settings).get();
188 | }
189 |
190 | return container;
191 | }
192 |
193 | public Builder() {
194 | container = new ESEmbededContainer();
195 | }
196 |
197 | /**
198 | * @param nodeName
199 | * @return Builder
200 | */
201 | public Builder withNodeName(String nodeName) {
202 | this.nodeName = nodeName;
203 | return this;
204 | }
205 |
206 | /**
207 | *
208 | * @param workingDir
209 | * @return Builder
210 | */
211 | public Builder withWorkingDir(String workingDir) {
212 | this.workingDir = workingDir;
213 | return this;
214 | }
215 |
216 | /**
217 | *
218 | * @param clusterName
219 | * @return Builder
220 | */
221 | public Builder withClusterName(String clusterName) {
222 | this.clusterName = clusterName;
223 | return this;
224 | }
225 |
226 | /**
227 | *
228 | * @param templateName
229 | * @param templateSource
230 | * @return Builder
231 | */
232 | public Builder withTemplate(String templateName, String templateSource) {
233 | this.templateName = templateName;
234 | this.templateSource = templateSource;
235 | return this;
236 | }
237 |
238 | /**
239 | *
240 | * @param snapshotWorkingLocation
241 | * @return Builder
242 | */
243 | public Builder withSnapshotWorkingLocation(String snapshotWorkingLocation) {
244 | this.snapshotWorkingLocation = snapshotWorkingLocation;
245 | return this;
246 | }
247 |
248 | /**
249 | *
250 | * @param snapshotRepoName
251 | * @return Builder
252 | */
253 | public Builder withSnapshotRepoName(String snapshotRepoName) {
254 | this.snapshotRepoName = snapshotRepoName;
255 | return this;
256 | }
257 |
258 | /**
259 | *
260 | * @param memoryBackedIndex
261 | * @return Builder
262 | */
263 | public Builder withInMemoryBackedIndexes(boolean memoryBackedIndex) {
264 | this.memoryBackedIndex = memoryBackedIndex;
265 | return this;
266 | }
267 |
268 | }
269 |
270 | /**
271 | *
272 | * @return Node
273 | */
274 | public Node getNode() {
275 | return node;
276 | }
277 |
278 | public void setNode(Node node) {
279 | this.node = node;
280 | }
281 |
282 | }
283 |
--------------------------------------------------------------------------------
/src/main/java/com/inin/analytics/elasticsearch/IndexingPostProcessor.java:
--------------------------------------------------------------------------------
1 | package com.inin.analytics.elasticsearch;
2 |
3 | import static org.elasticsearch.common.settings.ImmutableSettings.settingsBuilder;
4 |
5 | import java.io.BufferedReader;
6 | import java.io.File;
7 | import java.io.FileNotFoundException;
8 | import java.io.IOException;
9 | import java.io.InputStreamReader;
10 | import java.io.PrintWriter;
11 | import java.util.ArrayList;
12 | import java.util.HashMap;
13 | import java.util.HashSet;
14 | import java.util.List;
15 | import java.util.Map;
16 | import java.util.Set;
17 |
18 | import org.apache.commons.io.FileUtils;
19 | import org.apache.hadoop.conf.Configuration;
20 | import org.apache.hadoop.fs.FileSystem;
21 | import org.apache.hadoop.fs.FileUtil;
22 | import org.apache.hadoop.fs.Path;
23 | import org.elasticsearch.cluster.metadata.IndexMetaData;
24 | import org.slf4j.Logger;
25 | import org.slf4j.LoggerFactory;
26 |
27 | import com.inin.analytics.elasticsearch.transport.BaseTransport;
28 | import com.inin.analytics.elasticsearch.transport.SnapshotTransportStrategy;
29 |
30 | public class IndexingPostProcessor {
31 | private static transient Logger logger = LoggerFactory.getLogger(IndexingPostProcessor.class);
32 |
33 | /**
34 | * The job output in HDFS is just a manifest of indicies generated by the Job. Why? S3 is eventually consistent in some
35 | * zones. That means if you try to list the indicies you just generated by this job, you might miss some. Instead, we
36 | * have the job spit out tiny manifests. This method merges them together, de-dupes them, and if there's any shards that
37 | * didn't get generated because they have no data it puts a placeholder empty shard in it's place to satisfy ElasticSearch.
38 | *
39 | * @param jobOutput
40 | * @param manifestFile
41 | * @param scratchDir
42 | * @param shardConfig
43 | * @param conf
44 | * @param reducerClass
45 | * @throws IOException
46 | * @throws InstantiationException
47 | * @throws IllegalAccessException
48 | */
49 | public void execute(Path jobOutput, Path manifestFile, String scratchDir, ShardConfig shardConfig, Configuration conf, Class extends BaseESReducer> reducerClass) throws IOException, InstantiationException, IllegalAccessException {
50 | FileSystem fs = FileSystem.get(conf);
51 | ESEmbededContainer esEmbededContainer = null;
52 | boolean rootManifestUploaded = false;
53 | try{
54 | Map numShardsGenerated = new HashMap();
55 |
56 | // Each reducer spits out it's own manifest file, merge em all together into 1 file
57 | FileUtil.copyMerge(fs, jobOutput, fs, manifestFile, false, conf, "");
58 |
59 | // Read the merged file, de-duping entries as it reads
60 | BufferedReader br = new BufferedReader(new InputStreamReader(fs.open(manifestFile)));
61 | String line;
62 | line=br.readLine();
63 | Set indicies = new HashSet<>();
64 | while (line != null){
65 | indicies.add(line);
66 | int count = numShardsGenerated.containsKey(line) ? numShardsGenerated.get(line) : 0;
67 | numShardsGenerated.put(line, count + 1);
68 | line=br.readLine();
69 | }
70 |
71 | File scratch = new File(scratchDir);
72 | if(!scratch.exists()) {
73 | // Make the dir if it doesn't exist
74 | scratch.mkdirs();
75 | } else {
76 | FileUtils.deleteDirectory(scratch);
77 | scratch.mkdirs();
78 | }
79 |
80 | esEmbededContainer = getESEmbededContainer(conf, reducerClass);
81 |
82 | String scratchFile = scratchDir + "manifest";
83 | PrintWriter writer = new PrintWriter(scratchFile, "UTF-8");
84 |
85 | // Create all the indexes
86 | for(String index : indicies) {
87 | esEmbededContainer.getNode().client().admin().indices().prepareCreate(index).setSettings(settingsBuilder()
88 | .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, shardConfig.getShardsForIndex(index))).get();
89 | }
90 |
91 | // Snapshot it
92 | List indexesToSnapshot = new ArrayList<>();
93 | indexesToSnapshot.addAll(indicies);
94 | esEmbededContainer.snapshot(indexesToSnapshot, BaseESReducer.SNAPSHOT_NAME, conf.get(ConfigParams.SNAPSHOT_REPO_NAME_CONFIG_KEY.toString()), null);
95 |
96 | for(String index : indicies) {
97 | try{
98 | placeMissingIndexes(BaseESReducer.SNAPSHOT_NAME, esEmbededContainer, conf, index, shardConfig, !rootManifestUploaded);
99 | // The root level manifests are the same on each one, so it need only be uploaded once
100 | rootManifestUploaded = true;
101 | } catch (FileNotFoundException | IllegalArgumentException e) {
102 | logger.error("Unable to include index " + index + " in the manifest because missing shards could not be generated", e);
103 | continue;
104 | }
105 |
106 | // Re-write the manifest to local disk
107 | writer.println(index);
108 | }
109 |
110 | // Clean up index from embedded instance
111 | for(String index : indicies) {
112 | esEmbededContainer.getNode().client().admin().indices().prepareDelete(index).execute();
113 | }
114 |
115 | writer.close();
116 |
117 | // Move the manifest onto HDFS
118 | fs.copyFromLocalFile(new Path(scratchFile), manifestFile);
119 | } finally {
120 | if(esEmbededContainer != null) {
121 | esEmbededContainer.getNode().close();
122 | while(!esEmbededContainer.getNode().isClosed());
123 | }
124 | FileUtils.deleteDirectory(new File(conf.get(ConfigParams.SNAPSHOT_WORKING_LOCATION_CONFIG_KEY.toString())));
125 | }
126 | }
127 |
128 | /**
129 | *
130 | * @param snapshotName
131 | * @param esEmbededContainer
132 | * @param conf
133 | * @param index
134 | * @param shardConfig
135 | * @param includeRootManifest
136 | * @throws IOException
137 | */
138 | public void placeMissingIndexes(String snapshotName, ESEmbededContainer esEmbededContainer, Configuration conf, String index, ShardConfig shardConfig, boolean includeRootManifest) throws IOException {
139 | BaseTransport transport = SnapshotTransportStrategy.get(conf.get(ConfigParams.SNAPSHOT_WORKING_LOCATION_CONFIG_KEY.toString()), conf.get(ConfigParams.SNAPSHOT_FINAL_DESTINATION.toString()));
140 | transport.placeMissingShards(snapshotName, index, shardConfig, includeRootManifest);
141 | }
142 |
143 | /**
144 | * Returns a ESEmbededContainer configured for some local indexing
145 | *
146 | * @param conf
147 | * @param reducerClass
148 | * @return
149 | * @throws IOException
150 | * @throws InstantiationException
151 | * @throws IllegalAccessException
152 | */
153 | private ESEmbededContainer getESEmbededContainer(Configuration conf, Class extends BaseESReducer> reducerClass) throws IOException, InstantiationException, IllegalAccessException {
154 | ESEmbededContainer esEmbededContainer = null;
155 | BaseESReducer red = reducerClass.newInstance();
156 | String templateName = red.getTemplateName();
157 | String templateJson = red.getTemplate();
158 | red.close();
159 |
160 | ESEmbededContainer.Builder builder = new ESEmbededContainer.Builder()
161 | .withNodeName("embededESTempLoaderNode")
162 | .withInMemoryBackedIndexes(true)
163 | .withWorkingDir(conf.get(ConfigParams.ES_WORKING_DIR.toString()))
164 | .withClusterName("bulkLoadPartition")
165 | .withSnapshotWorkingLocation(conf.get(ConfigParams.SNAPSHOT_WORKING_LOCATION_CONFIG_KEY.toString()))
166 | .withSnapshotRepoName(conf.get(ConfigParams.SNAPSHOT_REPO_NAME_CONFIG_KEY.toString()));
167 |
168 | if(templateName != null && templateJson != null) {
169 | builder.withTemplate(templateName, templateJson);
170 | }
171 |
172 | esEmbededContainer = builder.build();
173 | return esEmbededContainer;
174 | }
175 | }
176 |
--------------------------------------------------------------------------------
/src/main/java/com/inin/analytics/elasticsearch/ShardConfig.java:
--------------------------------------------------------------------------------
1 | package com.inin.analytics.elasticsearch;
2 |
3 | import java.util.HashMap;
4 | import java.util.Map;
5 |
6 | /**
7 | * While many ES clusters have a uniform number of shards per index, if your
8 | * indexing volume ebbs & flows it makes sense to vary the shard count from day
9 | * to day. Given that you're rebuilding indices for historical data one usage pattern
10 | * has a script reach out to the ES cluster to retrieve the size of each index so
11 | * a shard count can be be calculated.
12 | *
13 | * For example, perhaps a desirable shard size is 10GB/shard. If the index on the cluster
14 | * currently takes 25GB, it could be calculated that 3 shards for said index would be appropriate.
15 | *
16 | * Note: Shard size metrics on live ES clusters aren't %100 accurate metric b/c it can vary during merge
17 | * operations, but the technique is still useful nonetheless. Historical data tends to be less mutable
18 | * leading to fewer merges. It's up to the implementer to determine an algorithm that fits their
19 | * implementation.
20 | *
21 | *
22 | * @author drew
23 | *
24 | */
25 | public class ShardConfig {
26 | private Map shardsPerIndex = new HashMap<>();
27 | private Map shardsPerOrg = new HashMap<>();
28 | private Long defaultShardsPerIndex = 5l;
29 | private Long defaultShardsPerOrg = 2l;
30 |
31 | public ShardConfig(Map shardsPerIndex, Map shardsPerOrg) {
32 | this.shardsPerIndex = shardsPerIndex;
33 | this.shardsPerOrg = shardsPerOrg;
34 | }
35 |
36 | public ShardConfig(Map shardsPerIndex, Map shardsPerOrg, Long defaultShardsPerIndex, Long defaultShardsPerOrg) {
37 | this.shardsPerIndex = shardsPerIndex;
38 | this.shardsPerOrg = shardsPerOrg;
39 | this.defaultShardsPerIndex = defaultShardsPerIndex;
40 | this.defaultShardsPerOrg = defaultShardsPerOrg;
41 | }
42 |
43 | public ShardConfig() {}
44 |
45 | public ShardConfig(Long defaultShardsPerIndex, Long defaultShardsPerOrg) {
46 | this.defaultShardsPerIndex = defaultShardsPerIndex;
47 | this.defaultShardsPerOrg = defaultShardsPerOrg;
48 | }
49 |
50 | public Long getShardsForIndex(String index) {
51 | if(shardsPerIndex.containsKey(index)) {
52 | return shardsPerIndex.get(index);
53 | }
54 |
55 | return defaultShardsPerIndex;
56 | }
57 | public Long getShardsForOrg(String index) {
58 | if(shardsPerOrg.containsKey(index)) {
59 | return shardsPerOrg.get(index);
60 | }
61 | return defaultShardsPerOrg;
62 | }
63 |
64 |
65 |
66 | @Override
67 | public String toString() {
68 | return "ShardConfig [shardsPerIndex=" + shardsPerIndex + ", shardsPerOrg=" + shardsPerOrg
69 | + ", defaultShardsPerIndex=" + defaultShardsPerIndex + ", defaultShardsPerOrg=" + defaultShardsPerOrg
70 | + "]";
71 | }
72 |
73 | }
74 |
--------------------------------------------------------------------------------
/src/main/java/com/inin/analytics/elasticsearch/driver/Driver.java:
--------------------------------------------------------------------------------
1 | package com.inin.analytics.elasticsearch.driver;
2 |
3 | import java.util.TimeZone;
4 |
5 | import org.apache.hadoop.util.ProgramDriver;
6 | import org.joda.time.DateTimeZone;
7 |
8 | import com.inin.analytics.elasticsearch.example.ExampleIndexingJob;
9 | import com.inin.analytics.elasticsearch.example.ExampleJobPrep;
10 | import com.inin.analytics.elasticsearch.example.GenerateData;
11 |
12 | public class Driver extends ProgramDriver {
13 |
14 | public Driver() throws Throwable {
15 | super();
16 | addClass("generateExampleData", GenerateData.class, "Example job for how to build documents for elasticsearch indexing");
17 | addClass("examplePrep", ExampleJobPrep.class, "Example job for how to build documents for elasticsearch indexing");
18 | addClass("esIndexRebuildExample", ExampleIndexingJob.class, "Example job for how to rebuild elasticsearch indexes");
19 | }
20 |
21 | public static void main(String[] args) throws Throwable {
22 | DateTimeZone.setDefault(DateTimeZone.UTC);
23 | TimeZone.setDefault(TimeZone.getTimeZone("GMT"));
24 | Driver driver = new Driver();
25 | driver.driver(args);
26 | System.exit(0);
27 | }
28 | }
29 |
--------------------------------------------------------------------------------
/src/main/java/com/inin/analytics/elasticsearch/example/ExampleIndexingJob.java:
--------------------------------------------------------------------------------
1 | package com.inin.analytics.elasticsearch.example;
2 |
3 | import org.apache.commons.lang.StringUtils;
4 | import org.apache.hadoop.conf.Configuration;
5 | import org.apache.hadoop.fs.Path;
6 | import org.apache.hadoop.io.Text;
7 | import org.apache.hadoop.mapred.FileInputFormat;
8 | import org.apache.hadoop.mapred.FileOutputFormat;
9 | import org.apache.hadoop.mapred.JobClient;
10 | import org.apache.hadoop.mapred.JobConf;
11 | import org.apache.hadoop.mapred.SequenceFileInputFormat;
12 | import org.apache.hadoop.mapred.TextOutputFormat;
13 | import org.apache.hadoop.util.Tool;
14 |
15 | import com.inin.analytics.elasticsearch.BaseESMapper;
16 | import com.inin.analytics.elasticsearch.ConfigParams;
17 | import com.inin.analytics.elasticsearch.IndexingPostProcessor;
18 | import com.inin.analytics.elasticsearch.ShardConfig;
19 |
20 | public class ExampleIndexingJob implements Tool {
21 |
22 | private static Configuration conf;
23 | public static int main(String[] args) throws Exception {
24 | if(args.length != 9) {
25 | System.err.println("Invalid # arguments. EG: loadES [pipe separated input] [snapshot working directory (fs/nfs)] [snapshot final destination (s3/nfs/hdfs)] [snapshot repo name] [elasticsearch working data location] [num reducers] [num shards per index] [num shards per organization] [manifest location]");
26 | return -1;
27 | }
28 |
29 | String inputPath = args[0];
30 | String snapshotWorkingLocation = args[1];
31 | String snapshotFinalDestination = args[2];
32 | String snapshotRepoName = args[3];
33 | String esWorkingDir = args[4];
34 | Integer numReducers = new Integer(args[5]);
35 | Long numShardsPerIndex = new Long(args[6]);
36 | Long numShardsPerOrganization = new Long(args[7]);
37 | String manifestLocation = args[8];
38 |
39 | // Remove trailing slashes from the destination
40 | snapshotFinalDestination = StringUtils.stripEnd(snapshotFinalDestination, "/");
41 |
42 | conf = new Configuration();
43 | conf.set(ConfigParams.SNAPSHOT_WORKING_LOCATION_CONFIG_KEY.toString(), snapshotWorkingLocation);
44 | conf.set(ConfigParams.SNAPSHOT_FINAL_DESTINATION.toString(), snapshotFinalDestination);
45 | conf.set(ConfigParams.SNAPSHOT_REPO_NAME_CONFIG_KEY.toString(), snapshotRepoName);
46 | conf.set(ConfigParams.ES_WORKING_DIR.toString(), esWorkingDir);
47 | conf.set(ConfigParams.NUM_SHARDS_PER_INDEX.toString(), numShardsPerIndex.toString());
48 | conf.set(ConfigParams.NUM_SHARDS_PER_ORGANIZATION.toString(), numShardsPerOrganization.toString());
49 |
50 | JobConf job = new JobConf(conf, ExampleIndexingJob.class);
51 | job.setJobName("Elastic Search Offline Index Generator");
52 | job.setInputFormat(SequenceFileInputFormat.class);
53 | job.setOutputFormat(TextOutputFormat.class);
54 | job.setMapperClass(BaseESMapper.class);
55 | job.setReducerClass(ExampleIndexingReducerImpl.class);
56 | job.setMapOutputValueClass(Text.class);
57 | job.setMapOutputKeyClass(Text.class);
58 | job.setNumReduceTasks(numReducers);
59 | job.setSpeculativeExecution(false);
60 |
61 | Path jobOutput = new Path(manifestLocation + "/raw/");
62 | Path manifestFile = new Path(manifestLocation + "manifest");
63 |
64 | FileOutputFormat.setOutputPath(job, jobOutput);
65 |
66 | // Set up inputs
67 | String[]inputFolders = StringUtils.split(inputPath, "|");
68 | for(String input : inputFolders) {
69 | FileInputFormat.addInputPath(job, new Path(input));
70 | }
71 |
72 | JobClient.runJob(job);
73 | IndexingPostProcessor postProcessor = new IndexingPostProcessor();
74 | postProcessor.execute(jobOutput, manifestFile, esWorkingDir, new ShardConfig(numShardsPerIndex, numShardsPerOrganization), conf, ExampleIndexingReducerImpl.class);
75 | return 0;
76 | }
77 |
78 | @Override
79 | public void setConf(Configuration conf) {
80 | this.conf = conf;
81 | }
82 |
83 | @Override
84 | public Configuration getConf() {
85 | return conf;
86 | }
87 |
88 | @Override
89 | public int run(String[] args) throws Exception {
90 | return ExampleIndexingJob.main(args);
91 | }
92 |
93 | }
--------------------------------------------------------------------------------
/src/main/java/com/inin/analytics/elasticsearch/example/ExampleIndexingReducerImpl.java:
--------------------------------------------------------------------------------
1 | package com.inin.analytics.elasticsearch.example;
2 |
3 | import org.apache.hadoop.mapred.JobConf;
4 |
5 | import com.inin.analytics.elasticsearch.BaseESReducer;
6 | import com.inin.analytics.elasticsearch.ConfigParams;
7 | import com.inin.analytics.elasticsearch.ShardConfig;
8 |
9 | public class ExampleIndexingReducerImpl extends BaseESReducer {
10 |
11 | /**
12 | * Provide the JSON contents of the index template. This is your hook for configuring ElasticSearch.
13 | *
14 | * http://www.elasticsearch.org/guide/en/elasticsearch/reference/current/indices-templates.html
15 | */
16 | public String getTemplate() {
17 | return null;
18 | }
19 |
20 | /**
21 | * Return a name for the index template.
22 | */
23 | @Override
24 | public String getTemplateName() {
25 | return null;
26 | }
27 |
28 | @Override
29 | public ShardConfig getShardConfig(JobConf job) {
30 | Long numShardsPerIndex = job.getLong(ConfigParams.NUM_SHARDS_PER_INDEX.name(), 5l);
31 | Long numShardsPerOrganization = job.getLong(ConfigParams.NUM_SHARDS_PER_ORGANIZATION.name(), 2l);
32 | return new ShardConfig(numShardsPerIndex, numShardsPerOrganization);
33 | }
34 |
35 | }
--------------------------------------------------------------------------------
/src/main/java/com/inin/analytics/elasticsearch/example/ExampleJobPrep.java:
--------------------------------------------------------------------------------
1 | package com.inin.analytics.elasticsearch.example;
2 |
3 | import java.io.IOException;
4 |
5 | import org.apache.commons.lang.StringUtils;
6 | import org.apache.hadoop.conf.Configuration;
7 | import org.apache.hadoop.fs.Path;
8 | import org.apache.hadoop.io.LongWritable;
9 | import org.apache.hadoop.io.Text;
10 | import org.apache.hadoop.mapreduce.Job;
11 | import org.apache.hadoop.mapreduce.Mapper;
12 | import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
13 | import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
14 | import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
15 | import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
16 | import org.apache.hadoop.util.Tool;
17 |
18 | import com.inin.analytics.elasticsearch.BaseESReducer;
19 | import com.inin.analytics.elasticsearch.index.rotation.ElasticSearchIndexMetadata;
20 | import com.inin.analytics.elasticsearch.index.routing.ElasticsearchRoutingStrategy;
21 | import com.inin.analytics.elasticsearch.index.routing.ElasticsearchRoutingStrategyV1;
22 |
23 | /**
24 | * Sample hadoop job for taking data from GenerateData.java and writes it out
25 | * into a format suitable for ExampleIndexingJob.java
26 | */
27 | public class ExampleJobPrep implements Tool {
28 | private static Configuration conf;
29 | private static final String INDEX_TYPE = "conversation";
30 | private static final String INDEX_SUFFIX_CONFIG = "indexSuffixConfigKey";
31 |
32 | private static final String NUM_SHARDS_PER_CUSTOMER = "numShardsPerCustomer";
33 | private static final String NUM_SHARDS = "numShards";
34 |
35 | public static class DocMapper extends Mapper {
36 | private ElasticsearchRoutingStrategy elasticsearchRoutingStrategy;
37 |
38 | @Override
39 | protected void setup(Context context) throws IOException, InterruptedException {
40 | Integer numShardsPerOrg = context.getConfiguration().getInt(NUM_SHARDS_PER_CUSTOMER, 1);
41 | Integer numShards = context.getConfiguration().getInt(NUM_SHARDS, 1);
42 |
43 | ElasticSearchIndexMetadata indexMetadata = new ElasticSearchIndexMetadata();
44 | indexMetadata.setNumShards(numShards);
45 | indexMetadata.setNumShardsPerOrg(numShardsPerOrg);
46 | elasticsearchRoutingStrategy = new ElasticsearchRoutingStrategyV1();
47 | elasticsearchRoutingStrategy.configure(indexMetadata);
48 | }
49 |
50 | @Override
51 | public void map(LongWritable key, Text value, Context context) throws IOException, InterruptedException {
52 | String[] csv = StringUtils.split(value.toString(), ",");
53 | String docId = csv[1];
54 | String customerId = csv[0];
55 | String json = "{\"customerId\":\"" + customerId + "\",\"color\":\"" + csv[2] + "\",\"id\":\"" + docId + "\",\"description\":\"" + csv[3] + "\"}";
56 | String routingHash = elasticsearchRoutingStrategy.getRoutingHash(customerId, docId);
57 |
58 | Text outputKey = new Text(INDEX_TYPE + BaseESReducer.TUPLE_SEPARATOR + routingHash);
59 | Text outputValue = new Text(INDEX_TYPE + BaseESReducer.TUPLE_SEPARATOR + customerId + BaseESReducer.TUPLE_SEPARATOR + json);
60 | context.write(outputKey, outputValue);
61 | }
62 | }
63 |
64 | public static boolean main(String[] args) throws Exception {
65 | if(args.length != 5) {
66 | System.err.println("Invalid # arguments. EG: loadES [pipe separated paths to source files containing segments & properties] [output location] [index name suffix] [numShardsPerIndex] [maxNumShardsPerCustomer (for routing)]");
67 | }
68 |
69 | String inputs = args[0];
70 | String output = args[1];
71 | String indexSuffix = args[2];
72 | Integer numShards = new Integer(args[3]);
73 | Integer numShardsPerCustomer = new Integer(args[4]);
74 |
75 | conf = new Configuration();
76 | Job job = Job.getInstance(conf, "Prep example");
77 | job.setJarByClass(ExampleJobPrep.class);
78 | job.setMapperClass(DocMapper.class);
79 | job.setOutputKeyClass(Text.class);
80 | job.setOutputValueClass(Text.class);
81 | job.setInputFormatClass(SequenceFileInputFormat.class);
82 | job.setOutputFormatClass(SequenceFileOutputFormat.class);
83 |
84 | job.setNumReduceTasks(0);
85 |
86 | job.getConfiguration().set(INDEX_SUFFIX_CONFIG, indexSuffix);
87 | job.getConfiguration().set(NUM_SHARDS_PER_CUSTOMER, numShardsPerCustomer.toString());
88 | job.getConfiguration().set(NUM_SHARDS, numShards.toString());
89 |
90 | FileOutputFormat.setOutputPath(job, new Path(output));
91 |
92 | // Set up inputs
93 | String[]inputFolders = StringUtils.split(inputs, "|");
94 | for(String input : inputFolders) {
95 | FileInputFormat.addInputPath(job, new Path(input));
96 | }
97 |
98 | return job.waitForCompletion(true);
99 | }
100 |
101 | @Override
102 | public void setConf(Configuration conf) {
103 | this.conf = conf;
104 | }
105 |
106 | @Override
107 | public Configuration getConf() {
108 | return conf;
109 | }
110 |
111 | @Override
112 | public int run(String[] args) throws Exception {
113 | boolean success = ExampleJobPrep.main(args);
114 | if(success) {
115 | return 0;
116 | } else {
117 | return 1;
118 | }
119 | }
120 |
121 | }
--------------------------------------------------------------------------------
/src/main/java/com/inin/analytics/elasticsearch/example/GenerateData.java:
--------------------------------------------------------------------------------
1 | package com.inin.analytics.elasticsearch.example;
2 |
3 | import java.util.UUID;
4 |
5 | import org.apache.commons.lang.RandomStringUtils;
6 | import org.apache.hadoop.conf.Configuration;
7 | import org.apache.hadoop.fs.FileSystem;
8 | import org.apache.hadoop.fs.Path;
9 | import org.apache.hadoop.io.LongWritable;
10 | import org.apache.hadoop.io.SequenceFile;
11 | import org.apache.hadoop.io.Text;
12 |
13 | public class GenerateData {
14 |
15 | public static void main(String[] args) throws Exception {
16 | if(args.length != 2) {
17 | System.err.println("Invalid # arguments. EG: [num sample records] [output file]");
18 | return;
19 | }
20 |
21 | Long numRecords = new Long(args[0]);
22 |
23 | String attribute = null;
24 | String customer = null;
25 | String customer1 = UUID.randomUUID().toString();
26 | String customer2 = UUID.randomUUID().toString();
27 |
28 | Configuration config = new Configuration();
29 | FileSystem fs = FileSystem.get(config);
30 | Path filenamePath = new Path(args[1]);
31 | SequenceFile.Writer inputWriter = new SequenceFile.Writer(fs, config, filenamePath, LongWritable.class, Text.class);
32 |
33 | for(Long x = 0l; x < numRecords; x++) {
34 | if(x % 2 == 0) {
35 | attribute = "yellow";
36 | } else {
37 | attribute = "blue";
38 | }
39 |
40 | if(x % 3 == 0) {
41 | customer = customer1;
42 | } else {
43 | customer = customer2;
44 | }
45 |
46 | inputWriter.append(new LongWritable(x), new Text(customer + "," + UUID.randomUUID().toString() + "," + attribute + "," + RandomStringUtils.randomAlphanumeric(15)));
47 | }
48 |
49 | inputWriter.close();
50 | }
51 | }
52 |
--------------------------------------------------------------------------------
/src/main/java/com/inin/analytics/elasticsearch/index/rotation/ElasticSearchIndexMetadata.java:
--------------------------------------------------------------------------------
1 | package com.inin.analytics.elasticsearch.index.rotation;
2 |
3 | import org.joda.time.LocalDate;
4 |
5 | import com.google.gson.annotations.Expose;
6 | import com.google.gson.annotations.SerializedName;
7 |
8 | /**
9 | * When an index has been rebuilt and we're substituting it in, this class holds onto
10 | * some metadata about the rebuilt index.
11 | *
12 | * @author drew
13 | *
14 | */
15 | public class ElasticSearchIndexMetadata {
16 |
17 | /**
18 | * Name of the index when it was first created
19 | */
20 |
21 | @Expose
22 | private String indexNameAtBirth;
23 |
24 | /**
25 | * Name of the index after it was rebuilt
26 | */
27 |
28 | @Expose
29 | private String rebuiltIndexName;
30 |
31 | /**
32 | * A shortened alias for the rebuilt index
33 | */
34 |
35 | @Expose
36 | private String rebuiltIndexAlias;
37 |
38 | /**
39 | * Date associated with the index (if applicable such as date partitioned data)
40 | */
41 |
42 | @SerializedName("indexLocalDate")
43 | @Expose
44 | private LocalDate indexDate;
45 |
46 | /**
47 | * Number if shards in the index
48 | */
49 |
50 | @Expose
51 | private int numShards;
52 |
53 | /**
54 | * Useful for the routing strategy, the number of shards an organization's data is split across within an index
55 | */
56 |
57 | @Expose
58 | private int numShardsPerOrg;
59 |
60 | /**
61 | * Class name for the routing strategy
62 | */
63 |
64 | @Expose
65 | private String routingStrategyClassName;
66 |
67 | public String getIndexNameAtBirth() {
68 | return indexNameAtBirth;
69 | }
70 | public void setIndexNameAtBirth(String indexNameAtBirth) {
71 | this.indexNameAtBirth = indexNameAtBirth;
72 | }
73 | public String getRebuiltIndexAlias() {
74 | return rebuiltIndexAlias;
75 | }
76 | public void setRebuiltIndexAlias(String rebuiltIndexAlias) {
77 | this.rebuiltIndexAlias = rebuiltIndexAlias;
78 | }
79 | public int getNumShardsPerOrg() {
80 | return numShardsPerOrg;
81 | }
82 | public void setNumShardsPerOrg(int numShardsPerOrg) {
83 | this.numShardsPerOrg = numShardsPerOrg;
84 | }
85 | public String getRoutingStrategyClassName() {
86 | return routingStrategyClassName;
87 | }
88 | public void setRoutingStrategyClassName(String routingStrategyClassName) {
89 | this.routingStrategyClassName = routingStrategyClassName;
90 | }
91 | public int getNumShards() {
92 | return numShards;
93 | }
94 | public void setNumShards(int numShards) {
95 | this.numShards = numShards;
96 | }
97 | public String getRebuiltIndexName() {
98 | return rebuiltIndexName;
99 | }
100 | public void setRebuiltIndexName(String rebuiltIndexName) {
101 | this.rebuiltIndexName = rebuiltIndexName;
102 | }
103 | public LocalDate getDate() {
104 | return indexDate;
105 | }
106 | public void setDate(LocalDate date) {
107 | this.indexDate = date;
108 | }
109 | @Override
110 | public String toString() {
111 | return "RotatedIndexMetadata [indexNameAtBirth=" + indexNameAtBirth
112 | + ", rebuiltIndexName=" + rebuiltIndexName
113 | + ", rebuiltIndexAlias=" + rebuiltIndexAlias + ", indexDate="
114 | + indexDate + ", numShards=" + numShards + ", numShardsPerOrg="
115 | + numShardsPerOrg + ", routingStrategyClassName="
116 | + routingStrategyClassName + "]";
117 | }
118 |
119 |
120 |
121 | }
122 |
--------------------------------------------------------------------------------
/src/main/java/com/inin/analytics/elasticsearch/index/rotation/ElasticsearchIndexRotationManager.java:
--------------------------------------------------------------------------------
1 | package com.inin.analytics.elasticsearch.index.rotation;
2 |
3 |
4 | /**
5 | * Swapping rebuilt indexes into an ES cluster with zero downtime requires holding
6 | * onto some state about the indexes. For example, if you rebuilt you cluster every night
7 | * then depending on data retention you could have multiple date stamped [on creation] versions
8 | * of an index. Here we keep track of what version is in use.
9 | *
10 | * @author drew
11 | *
12 | */
13 | public interface ElasticsearchIndexRotationManager {
14 |
15 | /**
16 | * Look up meta data for an index based on the original name it was created with
17 | * @param indexNameAtBirth
18 | * @return ElasticSearchIndexMetadata
19 | */
20 | ElasticSearchIndexMetadata getElasticSearchIndexMetadata(String indexNameAtBirth);
21 |
22 | /**
23 | * Once a rebuilt index has been loaded into ElasticSearch, this registers the metadata
24 | * with zookeeper so that other parts of the system know they can start using the new index.
25 | * @param rotatedIndexMetadata
26 | */
27 | void registerIndexAvailableOnRotation(ElasticSearchIndexMetadata rotatedIndexMetadata);
28 |
29 |
30 | /**
31 | * Optional: When rebuilding indexes, hold onto the state of the rebuild process. This is
32 | * useful if you wish to defer writes to an index being rebuilt until it's done and swapped in.
33 | *
34 | * @param state
35 | */
36 | void updateRebuildPipelineState(RebuildPipelineState state);
37 |
38 | /**
39 | * Get the current state of the index rebuild process. Again, this is optional and relies on updateRebuildPipelineState being used.
40 | *
41 | * @return RebuildPipelineState
42 | */
43 | RebuildPipelineState getRebuildPipelineState();
44 | }
45 |
--------------------------------------------------------------------------------
/src/main/java/com/inin/analytics/elasticsearch/index/rotation/ElasticsearchIndexRotationManagerNoop.java:
--------------------------------------------------------------------------------
1 | package com.inin.analytics.elasticsearch.index.rotation;
2 |
3 |
4 |
5 | public class ElasticsearchIndexRotationManagerNoop implements ElasticsearchIndexRotationManager {
6 |
7 |
8 | @Override
9 | public void registerIndexAvailableOnRotation(ElasticSearchIndexMetadata rotatedIndexMetadata) {
10 |
11 | }
12 |
13 | @Override
14 | public void updateRebuildPipelineState(RebuildPipelineState state) {
15 |
16 | }
17 |
18 | @Override
19 | public RebuildPipelineState getRebuildPipelineState() {
20 | return RebuildPipelineState.COMPLETE;
21 | }
22 |
23 | @Override
24 | public ElasticSearchIndexMetadata getElasticSearchIndexMetadata(String indexNameAtBirth) {
25 | ElasticSearchIndexMetadata rotatedIndexMetadata = new ElasticSearchIndexMetadata();
26 | rotatedIndexMetadata.setIndexNameAtBirth(indexNameAtBirth);
27 | return rotatedIndexMetadata;
28 | }
29 |
30 | }
31 |
--------------------------------------------------------------------------------
/src/main/java/com/inin/analytics/elasticsearch/index/rotation/ElasticsearchIndexRotationManagerZookeeper.java:
--------------------------------------------------------------------------------
1 | package com.inin.analytics.elasticsearch.index.rotation;
2 |
3 | import java.util.HashMap;
4 | import java.util.Map;
5 | import java.util.Map.Entry;
6 |
7 | import org.apache.commons.lang.StringUtils;
8 | import org.apache.curator.framework.CuratorFramework;
9 | import org.apache.curator.framework.listen.Listenable;
10 | import org.apache.curator.framework.recipes.cache.ChildData;
11 | import org.apache.curator.framework.recipes.cache.NodeCache;
12 | import org.apache.curator.framework.state.ConnectionState;
13 | import org.apache.curator.framework.state.ConnectionStateListener;
14 | import org.apache.zookeeper.KeeperException;
15 | import org.apache.zookeeper.KeeperException.Code;
16 | import org.slf4j.Logger;
17 | import org.slf4j.LoggerFactory;
18 |
19 | import com.google.common.base.Preconditions;
20 | import com.google.gson.Gson;
21 | import com.inin.analytics.elasticsearch.util.GsonFactory;
22 |
23 | public abstract class ElasticsearchIndexRotationManagerZookeeper implements ElasticsearchIndexRotationManager {
24 |
25 | protected abstract String getBaseZnode();
26 | protected abstract String getRebuildStateZnode();
27 |
28 | protected CuratorFramework curator;
29 |
30 | protected static transient Logger logger = LoggerFactory.getLogger(ElasticsearchIndexRotationManagerZookeeper.class);
31 | protected Map indexNameCache = new HashMap<>();
32 | protected NodeCache rebuildStateCache;
33 | protected Gson gson;
34 | protected static final String FAIL_MESSAGE = "Failed getting routing strategy from zookeeper for ";
35 | protected Listenable connectionStateListener;
36 |
37 |
38 |
39 | public void setCurator(CuratorFramework curator) {
40 | this.curator = curator;
41 | }
42 |
43 | public void init() {
44 | Preconditions.checkNotNull(curator, "curator is a required dependency");
45 | gson = GsonFactory.buildGsonBuilder().excludeFieldsWithoutExposeAnnotation().create();
46 |
47 | // AR-1785 Create watcher to rebuild nodeCache after ZK reconnects from a connection blip
48 | connectionStateListener = curator.getConnectionStateListenable();
49 | connectionStateListener.addListener(new ConnectionStateListener() {
50 | @Override
51 | public void stateChanged(CuratorFramework curator, ConnectionState state) {
52 | if (state.equals(ConnectionState.RECONNECTED) && indexNameCache != null) {
53 | for(Entry nodeCachePair : indexNameCache.entrySet()) {
54 | try {
55 | // nodeCache stops updating after a connection blip nukes its listener. I'd almost consider that a bug in curator, but for now this is the advised workaround.
56 | logger.info("ZK connection reconnect detected, rebuilding curator nodeCache for " + nodeCachePair.getKey());
57 | nodeCachePair.getValue().rebuild();
58 | } catch (Exception e) {
59 | logger.warn("Failed to rebuild nodeCache after ZK reconnect ", e);
60 | }
61 | }
62 | }
63 | }
64 | });
65 | }
66 |
67 | protected void ensureNodePathExists(String zkPath) throws Exception {
68 | try {
69 | String[] pieces = StringUtils.split(zkPath, "/");
70 | String znode = "";
71 | for(String piece : pieces) {
72 | znode = znode + "/" + piece;
73 | try {
74 | curator.getData().forPath(znode);
75 | } catch (KeeperException e) {
76 | Code errorCode = e.code();
77 | if(errorCode.equals(Code.NONODE)) {
78 | curator.create().forPath(znode);
79 | }
80 | }
81 | }
82 | } catch (Exception e) {
83 | throw new IllegalStateException("Failed to initialize ElasticsearchIndexRotationStrategy because base ZK nodes could not be created", e);
84 | }
85 | }
86 |
87 |
88 | /**
89 | * Register that an index is on the hadoop rebuild train. We'll store the index name that the live data
90 | * would normally flow into with the alias to the rebuilt index. EG
91 | *
92 | * New index at birth looks like
93 | * c140101
94 | *
95 | * Rebuilt index
96 | * c140101_reubild_82389238923
97 | *
98 | * Rebuilt index alias
99 | * c140101_r
100 | *
101 | * @param bucket
102 | */
103 |
104 | @Override
105 | public void registerIndexAvailableOnRotation(ElasticSearchIndexMetadata rotatedIndexMetadata) {
106 | String indexNameZnode = getBaseZnode() + rotatedIndexMetadata.getIndexNameAtBirth();
107 | try {
108 | ensureNodePathExists(indexNameZnode);
109 | // Persisting metadata in json
110 | curator.setData().forPath(indexNameZnode, gson.toJson(rotatedIndexMetadata).getBytes());
111 | } catch (Exception e) {
112 | throw new IllegalStateException("Unable to register znode " + indexNameZnode, e);
113 | }
114 | }
115 |
116 | /**
117 | * @param String
118 | * @return ElasticSearchIndexMetadata
119 | */
120 | public ElasticSearchIndexMetadata getElasticSearchIndexMetadata(String indexNameAtBirth) {
121 | String znode = getBaseZnode() + indexNameAtBirth;
122 | try {
123 | if(!indexNameCache.containsKey(indexNameAtBirth)) {
124 | createNodeCacheForName(znode, indexNameAtBirth);
125 | }
126 | ChildData cd = indexNameCache.get(indexNameAtBirth).getCurrentData();
127 |
128 | if(cd != null) {
129 | String json = new String(cd.getData());
130 | return gson.fromJson(json, ElasticSearchIndexMetadata.class);
131 | }
132 | } catch (Exception e) {
133 | throw new IllegalStateException("Error retrieving znode, unable to maintain index metadata ", e);
134 | }
135 |
136 | ElasticSearchIndexMetadata metadata = new ElasticSearchIndexMetadata();
137 | metadata.setIndexNameAtBirth(indexNameAtBirth);
138 | return metadata;
139 | }
140 |
141 |
142 | protected synchronized void createNodeCacheForName(String zkPath, final String indexName) throws Exception {
143 | final NodeCache nodeCache = new NodeCache(curator, zkPath);
144 | nodeCache.start(true);
145 | indexNameCache.put(indexName, nodeCache);
146 | }
147 |
148 | @Override
149 | public void updateRebuildPipelineState(RebuildPipelineState state) {
150 | try {
151 | ensureNodePathExists(getRebuildStateZnode());
152 | // Persisting metadata in json
153 | curator.setData().forPath(getRebuildStateZnode(), state.name().getBytes());
154 | } catch (Exception e) {
155 | throw new IllegalStateException("Unable to register state " + state, e);
156 | }
157 | }
158 |
159 | /**
160 | * @return RebuildPipelineState
161 | */
162 | @Override
163 | public RebuildPipelineState getRebuildPipelineState() {
164 | if(rebuildStateCache == null) {
165 | rebuildStateCache = new NodeCache(curator, getRebuildStateZnode());
166 | try {
167 | rebuildStateCache.start(true);
168 | } catch (Exception e) {
169 | throw new IllegalStateException("Unable to get pipeline rebuild state", e);
170 | }
171 | }
172 | ChildData cd = rebuildStateCache.getCurrentData();
173 | if(cd != null) {
174 | return RebuildPipelineState.valueOf(new String(cd.getData()));
175 | } else {
176 | // COMPLETE ~= NOT_RUNNING, so if it's never been ran that's what we default to. At some point we'll want to fix the enum.
177 | return RebuildPipelineState.COMPLETE;
178 | }
179 | }
180 |
181 |
182 | }
183 |
--------------------------------------------------------------------------------
/src/main/java/com/inin/analytics/elasticsearch/index/rotation/ExampleElasticsearchIndexRotationStrategyZookeeper.java:
--------------------------------------------------------------------------------
1 | package com.inin.analytics.elasticsearch.index.rotation;
2 |
3 | public class ExampleElasticsearchIndexRotationStrategyZookeeper extends ElasticsearchIndexRotationManagerZookeeper {
4 | private static final String INDEX_NAME_BASE_ZNODE = "/example/index/alias/";
5 | private static final String INDEX_REBUILD_PIPELINE_STATE_ZNODE = "/example/index/rebuild/pipeline/state";
6 |
7 | @Override
8 | protected String getBaseZnode() {
9 | return INDEX_NAME_BASE_ZNODE;
10 | }
11 | @Override
12 | protected String getRebuildStateZnode() {
13 | return INDEX_REBUILD_PIPELINE_STATE_ZNODE;
14 | }
15 | }
--------------------------------------------------------------------------------
/src/main/java/com/inin/analytics/elasticsearch/index/rotation/RebuildPipelineState.java:
--------------------------------------------------------------------------------
1 | package com.inin.analytics.elasticsearch.index.rotation;
2 |
3 | public enum RebuildPipelineState {
4 | RUNNING, // Pipeline is running
5 | COMPLETE // Pipeline is done running
6 | }
7 |
--------------------------------------------------------------------------------
/src/main/java/com/inin/analytics/elasticsearch/index/routing/ElasticsearchRoutingStrategy.java:
--------------------------------------------------------------------------------
1 | package com.inin.analytics.elasticsearch.index.routing;
2 |
3 | import com.inin.analytics.elasticsearch.index.rotation.ElasticSearchIndexMetadata;
4 |
5 | public interface ElasticsearchRoutingStrategy extends java.io.Serializable {
6 | String getRoutingHash(String orgId, String convId);
7 | String[] getPossibleRoutingHashes(String orgId);
8 | void configure(ElasticSearchIndexMetadata rotatedIndexMetadata);
9 | int getNumShardsPerOrg();
10 | int getNumShards();
11 | }
12 |
--------------------------------------------------------------------------------
/src/main/java/com/inin/analytics/elasticsearch/index/routing/ElasticsearchRoutingStrategyV1.java:
--------------------------------------------------------------------------------
1 | package com.inin.analytics.elasticsearch.index.routing;
2 |
3 | import java.util.HashMap;
4 | import java.util.Map;
5 |
6 | import com.google.common.base.Preconditions;
7 | import com.inin.analytics.elasticsearch.index.rotation.ElasticSearchIndexMetadata;
8 | import com.inin.analytics.elasticsearch.util.MurmurHash;
9 |
10 |
11 | /**
12 | * This routing strategy for elasticsearch. Read up on what routing does here
13 | * http://www.elasticsearch.org/blog/customizing-your-document-routing/
14 | *
15 | * Perhaps you have 10 shards per index and you don't wish to query every shard
16 | * every time you do a search against an org. A simple sharding strategy would
17 | * put all the data for 1 org on 1 shard using consistant hashing on orgId. However that
18 | * has the potential to hotspot some shards if an org with a lot of data comes
19 | * through.
20 | *
21 | * This attempts to alleviate that by making the # subset of shards configurable. EG
22 | * numShards = 10, numShardsPerOrg = 3, all of an org's data will be split to one of
23 | * 3 shards. Which one of the 3 is determined by hashing the conversationId.
24 | *
25 | * Note: DO NOT CHANGE THIS CLASS. It's immutable once it's been used to generate ES indexes
26 | * so changing it affects data routing and will make data appear unavailable b/c its looking
27 | * in the wrong shard. The correct thing to do is to make a newer version of this class,
28 | * say ElasticsearchRoutingStrategyV2 and see to it that the hadoop jobs to rebuild
29 | * the ES indexes not only use it, but update zookeeper with which implementation
30 | * indexes were built with. That way you can evolve the routing strategy without breaking
31 | * anything.
32 | *
33 | * @author drew
34 | *
35 | */
36 | public class ElasticsearchRoutingStrategyV1 implements ElasticsearchRoutingStrategy, java.io.Serializable {
37 | private static final long serialVersionUID = 1L;
38 | private int numShardsPerOrg;
39 | private int numShards;
40 | private Map shardToRout = new HashMap<>();
41 |
42 | /**
43 | * Adapted from DjbHashFunction & PlainOperationRouting in Elasticsearch. This is the default hashing
44 | * Algorithm for doc routing. We need this to reverse engineer routing strings that rout to
45 | * the shard we want.
46 | *
47 | * @param value
48 | * @return
49 | */
50 | public int hash(String value) {
51 | long hash = 5381;
52 | for (int i = 0; i < value.length(); i++) {
53 | hash = ((hash << 5) + hash) + value.charAt(i);
54 | }
55 |
56 | return Math.abs((int) hash % numShards);
57 | }
58 |
59 | public void init() {
60 | Integer x = 0;
61 | while(shardToRout.size() < numShards) {
62 | Integer hash = hash(x.toString());
63 | if(shardToRout.get(x) == null) {
64 | shardToRout.put(x, hash);
65 | }
66 | x++;
67 | }
68 | }
69 |
70 | public ElasticsearchRoutingStrategyV1() {
71 |
72 | }
73 |
74 | @Override
75 | public void configure(ElasticSearchIndexMetadata rotatedIndexMetadata) {
76 | Preconditions.checkNotNull(rotatedIndexMetadata.getNumShardsPerOrg(), "Num shards per org must not be null with " + this.getClass().getSimpleName());
77 | Preconditions.checkNotNull(rotatedIndexMetadata.getNumShards(), "Num shards must not be null with " + this.getClass().getSimpleName());
78 | this.numShardsPerOrg = rotatedIndexMetadata.getNumShardsPerOrg();
79 | this.numShards = rotatedIndexMetadata.getNumShards();
80 | init();
81 | }
82 |
83 | @Override
84 | public int getNumShardsPerOrg() {
85 | return numShardsPerOrg;
86 | }
87 |
88 | @Override
89 | public int getNumShards() {
90 | return numShards;
91 | }
92 |
93 |
94 | public Map getShardToRout() {
95 | return shardToRout;
96 | }
97 |
98 | /**
99 | * For an orgId & convId, get the shard routing for a document.
100 | *
101 | * Note: ES re-hashes routing values so shard 1 wont necessarily mean
102 | * your data ends up in shard 1. However, if you realize that
103 | * then you're in a bad place.
104 | *
105 | *
106 | * @param orgId
107 | * @param convId
108 | * @param numShards
109 | * @param numShardsPerOrg
110 | * @return
111 | */
112 |
113 | @Override
114 | public String getRoutingHash(String orgId, String convId) {
115 | Preconditions.checkArgument(numShards >= numShardsPerOrg, "Misconfigured, numShards must be >= numShardsPerOrg");
116 | int orgIdHash = getOrgIdHash(orgId, numShards);
117 | int convIdHash = Math.abs(MurmurHash.getInstance().hash(convId.getBytes(), 0)) % numShardsPerOrg;
118 |
119 | int shard = orgIdHash + convIdHash;
120 | while(shard >= numShards) {
121 | shard -= numShards;
122 | }
123 |
124 | return shardToRout.get(shard).toString();
125 | }
126 |
127 | /**
128 | * When searching data for an Org, you may desire to only search the shards
129 | * which hold data for that Org. This gives you a list of possible shard routings.
130 | *
131 | * @param orgId
132 | * @param numShards
133 | * @param numShardsPerOrg
134 | * @return
135 | */
136 | @Override
137 | public String[] getPossibleRoutingHashes(String orgId) {
138 | int orgIdHash = getOrgIdHash(orgId, numShards);
139 | String[] possibleShards = new String[numShardsPerOrg];
140 | for(int x = 0; x < numShardsPerOrg; x ++) {
141 | int shard = orgIdHash + x;
142 | while(shard >= numShards) {
143 | shard -= numShards;
144 | }
145 | possibleShards[x] = shardToRout.get(shard).toString();
146 | }
147 | return possibleShards;
148 | }
149 |
150 | private int getOrgIdHash(String orgId, int numShards) {
151 | return Math.abs(MurmurHash.getInstance().hash(orgId.getBytes(), 0)) % numShards;
152 | }
153 |
154 | @Override
155 | public int hashCode() {
156 | final int prime = 31;
157 | int result = 1;
158 | result = prime * result + numShards;
159 | result = prime * result + numShardsPerOrg;
160 | return result;
161 | }
162 |
163 | @Override
164 | public boolean equals(Object obj) {
165 | if (this == obj)
166 | return true;
167 | if (obj == null)
168 | return false;
169 | if (getClass() != obj.getClass())
170 | return false;
171 | ElasticsearchRoutingStrategyV1 other = (ElasticsearchRoutingStrategyV1) obj;
172 | if (numShards != other.numShards)
173 | return false;
174 | if (numShardsPerOrg != other.numShardsPerOrg)
175 | return false;
176 | return true;
177 | }
178 |
179 | @Override
180 | public String toString() {
181 | return "ElasticsearchRoutingStrategyV1 [numShardsPerOrg=" + numShardsPerOrg + ", numShards=" + numShards
182 | + ", shardToRout=" + shardToRout + "]";
183 | }
184 |
185 |
186 | }
187 |
--------------------------------------------------------------------------------
/src/main/java/com/inin/analytics/elasticsearch/index/selector/RealtimeIndexSelectionStrategy.java:
--------------------------------------------------------------------------------
1 | package com.inin.analytics.elasticsearch.index.selector;
2 |
3 | import java.util.List;
4 | import java.util.Set;
5 |
6 | import com.inin.analytics.elasticsearch.index.rotation.ElasticSearchIndexMetadata;
7 | import com.inin.analytics.elasticsearch.index.routing.ElasticsearchRoutingStrategy;
8 |
9 | public interface RealtimeIndexSelectionStrategy {
10 | ElasticsearchRoutingStrategy get(ElasticSearchIndexMetadata rotatedIndexMetadata);
11 | Set getRoutingStrategiesForIndicies(List indices);
12 |
13 | @Deprecated // Use getRoutingStrategiesForIndicies
14 | ElasticsearchRoutingStrategy getRoutingStrategyForIndicies(List indices);
15 | String getIndexWritable(ElasticSearchIndexMetadata rotatedIndexMetadata);
16 | String getIndexReadable(ElasticSearchIndexMetadata rotatedIndexMetadata);
17 |
18 | }
19 |
--------------------------------------------------------------------------------
/src/main/java/com/inin/analytics/elasticsearch/index/selector/RealtimeIndexSelectionStrategyLagged.java:
--------------------------------------------------------------------------------
1 | package com.inin.analytics.elasticsearch.index.selector;
2 |
3 | import java.util.HashSet;
4 | import java.util.List;
5 | import java.util.Set;
6 |
7 | import org.joda.time.DateTime;
8 | import org.slf4j.Logger;
9 | import org.slf4j.LoggerFactory;
10 |
11 | import com.inin.analytics.elasticsearch.index.rotation.ElasticSearchIndexMetadata;
12 | import com.inin.analytics.elasticsearch.index.routing.ElasticsearchRoutingStrategy;
13 |
14 |
15 | /**
16 | * For a realtime engine feeding data into elasticsearch or pulling data out, some strategies
17 | * around which index to use are employed here. For example, we know if live data is streaming
18 | * into today's index that we may want to ignore today's index that got rebuilt in hadoop because
19 | * the data invariably changed from the second the job started. This employs an x day lag on
20 | * switching over reads/writes to hadoop generated indexes.
21 | *
22 | * @author drew
23 | *
24 | */
25 | public class RealtimeIndexSelectionStrategyLagged implements RealtimeIndexSelectionStrategy {
26 |
27 | private int LAG = 2;
28 | protected static final String FAIL_MESSAGE = "Failed getting routing strategy from zookeeper for ";
29 | private static transient Logger logger = LoggerFactory.getLogger(RealtimeIndexSelectionStrategyLagged.class);
30 |
31 |
32 | /**
33 | *
34 | * @param lAG
35 | */
36 | public RealtimeIndexSelectionStrategyLagged(int lAG) {
37 | super();
38 | LAG = lAG;
39 | }
40 |
41 | public ElasticsearchRoutingStrategy get(ElasticSearchIndexMetadata rotatedIndexMetadata) {
42 |
43 | DateTime now = new DateTime();
44 | if(rotatedIndexMetadata != null && rotatedIndexMetadata.getRoutingStrategyClassName() != null && !rotatedIndexMetadata.getDate().isAfter(now.minusDays(LAG).toLocalDate())) {
45 | ElasticsearchRoutingStrategy strategy;
46 | try {
47 | strategy = (ElasticsearchRoutingStrategy) Class.forName(rotatedIndexMetadata.getRoutingStrategyClassName()).newInstance();
48 | strategy.configure(rotatedIndexMetadata);
49 | return strategy;
50 | } catch (InstantiationException e) {
51 | logger.error(FAIL_MESSAGE + rotatedIndexMetadata, e);
52 | } catch (IllegalAccessException e) {
53 | logger.error(FAIL_MESSAGE + rotatedIndexMetadata, e);
54 | } catch (ClassNotFoundException e) {
55 | logger.error(FAIL_MESSAGE + rotatedIndexMetadata, e);
56 | }
57 | }
58 | return null;
59 | }
60 |
61 | /**
62 | * Check if the routing strategy is the same for all the indicies. This is useful
63 | * when searching because all indexes have to have the same routing strategy in order to
64 | * cut down the # of shards to search. If they differ, then you have to hit the whole index.
65 | *
66 | * Returns null if there's no common routing strategy
67 | *
68 | * @param indices
69 | * @return
70 | */
71 | public Set getRoutingStrategiesForIndicies(List indices) {
72 | Set strategies = new HashSet<>();
73 | ElasticsearchRoutingStrategy routingStrategy = null;
74 | for(ElasticSearchIndexMetadata index : indices) {
75 | if(index.getRoutingStrategyClassName() == null) {
76 | // If the routing strategy isn't set, then there can be no common strategy
77 | routingStrategy = null;
78 | break;
79 | } else if(routingStrategy == null) {
80 | strategies.add(get(index));
81 | } else {
82 | ElasticsearchRoutingStrategy routingStrategy2 = get(index);
83 | if(routingStrategy2 == null || !routingStrategy2.equals(routingStrategy)) {
84 | routingStrategy = null;
85 | break;
86 | }
87 | }
88 | }
89 | return strategies;
90 | }
91 |
92 | /**
93 | * Warning, this method only works if all indices contain the same number of shards. Please
94 | * move to getRoutingStrategiesForIndicies(..)
95 | */
96 | @Deprecated
97 | @Override
98 | public ElasticsearchRoutingStrategy getRoutingStrategyForIndicies(List indices) {
99 | ElasticsearchRoutingStrategy routingStrategy = null;
100 | for(ElasticSearchIndexMetadata index : indices) {
101 | if(index.getRoutingStrategyClassName() == null) {
102 | // If the routing strategy isn't set, then there can be no common strategy
103 | routingStrategy = null;
104 | break;
105 | } else if(routingStrategy == null) {
106 | routingStrategy = get(index);
107 | } else {
108 | ElasticsearchRoutingStrategy routingStrategy2 = get(index);
109 | if(routingStrategy2 == null || !routingStrategy2.equals(routingStrategy)) {
110 | routingStrategy = null;
111 | break;
112 | }
113 | }
114 | }
115 | return routingStrategy;
116 | }
117 |
118 | public String getIndexWritable(ElasticSearchIndexMetadata rotatedIndexMetadata) {
119 | DateTime now = new DateTime();
120 | if(rotatedIndexMetadata.getRebuiltIndexAlias() == null || (rotatedIndexMetadata.getDate() != null && rotatedIndexMetadata.getDate().isAfter(now.minusDays(LAG).toLocalDate()))) {
121 | // Only use rotated indexes for data that's ROTATION_LAG_DAYS old
122 | return rotatedIndexMetadata.getIndexNameAtBirth();
123 | }
124 |
125 | if(rotatedIndexMetadata.getRebuiltIndexName() == null) {
126 | // For backwards compatibility (we didn't always write the rebuiltIndexName to zookeeper), we'll fall back on the alias
127 | return rotatedIndexMetadata.getRebuiltIndexAlias();
128 | }
129 |
130 | // Ideally you write to the index by it's full name
131 | return rotatedIndexMetadata.getRebuiltIndexName();
132 | }
133 |
134 | /**
135 | * When reading from ES, searches may hit many indexes. To help avoid hitting the URL size limit
136 | * when doing searches that hit a large # of indexes, we alias them with something short.
137 | */
138 | public String getIndexReadable(ElasticSearchIndexMetadata rotatedIndexMetadata) {
139 | DateTime now = new DateTime();
140 | if(rotatedIndexMetadata.getRebuiltIndexAlias() == null || (rotatedIndexMetadata.getDate() != null && rotatedIndexMetadata.getDate().isAfter(now.minusDays(LAG).toLocalDate()))) {
141 | // Only use rotated indexes for data that's ROTATION_LAG_DAYS old
142 | return rotatedIndexMetadata.getIndexNameAtBirth();
143 | }
144 |
145 | return rotatedIndexMetadata.getRebuiltIndexAlias();
146 | }
147 |
148 | }
149 |
--------------------------------------------------------------------------------
/src/main/java/com/inin/analytics/elasticsearch/transport/BaseTransport.java:
--------------------------------------------------------------------------------
1 | package com.inin.analytics.elasticsearch.transport;
2 |
3 | import java.io.File;
4 | import java.io.FilenameFilter;
5 | import java.io.IOException;
6 |
7 | import org.apache.commons.io.FileUtils;
8 |
9 | import com.google.common.base.Preconditions;
10 | import com.inin.analytics.elasticsearch.BaseESReducer;
11 | import com.inin.analytics.elasticsearch.ShardConfig;
12 | import com.inin.analytics.elasticsearch.transport.SnapshotTransportStrategy.STORAGE_SYSTEMS;
13 |
14 | public abstract class BaseTransport {
15 | protected String snapshotWorkingLocation;
16 | protected String snapshotFinalDestination;
17 | private DirectoryFilter directoryFilter = new DirectoryFilter();
18 |
19 | public BaseTransport(String snapshotWorkingLocation, String snapshotFinalDestination) {
20 | this.snapshotWorkingLocation = snapshotWorkingLocation;
21 | this.snapshotFinalDestination = snapshotFinalDestination;
22 | Preconditions.checkNotNull(snapshotWorkingLocation);
23 | Preconditions.checkNotNull(snapshotFinalDestination);
24 | }
25 |
26 | protected abstract void init();
27 | protected abstract void close();
28 | protected abstract void transferFile(boolean deleteSource, String destination, String filename, String localDirectory) throws IOException;
29 | protected abstract void transferDir(String destination, String localShardPath, String shard) throws IOException;
30 | protected abstract boolean checkExists(String destination, Integer shardNumber) throws IOException;
31 |
32 | /**
33 | * Transport a snapshot sitting on the local filesystem to a remote repository. Snapshots are stiched together
34 | * shard by shard because we're snapshotting 1 shard at a time.
35 | *
36 | * @param snapshotName
37 | * @param index
38 | * @param shardNumber
39 | * @throws IOException
40 | */
41 | public void execute(String snapshotName, String index) throws IOException {
42 | init();
43 | // Figure out which shard has all the data
44 | String largestShard = getShardSource(index);
45 |
46 | String destination = removeStorageSystemFromPath(snapshotFinalDestination);
47 |
48 | // Upload top level manifests
49 | transferFile(false, destination, "metadata-" + snapshotName, snapshotWorkingLocation);
50 | transferFile(false, destination, "snapshot-" + snapshotName, snapshotWorkingLocation);
51 | transferFile(false, destination, "index", snapshotWorkingLocation);
52 |
53 |
54 | // Upload per-index manifests
55 | String indexManifestSource = snapshotWorkingLocation + "indices" + BaseESReducer.DIR_SEPARATOR + index;
56 | String indexManifestDestination = destination + BaseESReducer.DIR_SEPARATOR + "indices" + BaseESReducer.DIR_SEPARATOR + index;
57 |
58 | transferFile(false, indexManifestDestination, "snapshot-" + snapshotName, indexManifestSource);
59 |
60 | // Cleanup shard data
61 | cleanEmptyShards(index, largestShard);
62 |
63 | // Upload shard data
64 | String shardSource = snapshotWorkingLocation + "indices" + BaseESReducer.DIR_SEPARATOR + index + BaseESReducer.DIR_SEPARATOR + largestShard;
65 |
66 | String shardDestination = destination + BaseESReducer.DIR_SEPARATOR + "indices" + BaseESReducer.DIR_SEPARATOR + index + BaseESReducer.DIR_SEPARATOR;
67 | transferDir(shardDestination, shardSource, largestShard);
68 | close();
69 | }
70 |
71 | public void placeMissingShards(String snapshotName, String index, ShardConfig shardConfig, boolean includeRootManifest) throws IOException {
72 | init();
73 | String destination = removeStorageSystemFromPath(snapshotFinalDestination);
74 |
75 | if(includeRootManifest) {
76 | // Upload top level manifests
77 | transferFile(false, destination, "metadata-" + snapshotName, snapshotWorkingLocation);
78 | transferFile(false, destination, "snapshot-" + snapshotName, snapshotWorkingLocation);
79 | transferFile(false, destination, "index", snapshotWorkingLocation);
80 | }
81 |
82 | for(int shard = 0; shard < shardConfig.getShardsForIndex(index); shard++) {
83 | String indexDestination = destination + BaseESReducer.DIR_SEPARATOR + "indices" + BaseESReducer.DIR_SEPARATOR + index + BaseESReducer.DIR_SEPARATOR ;
84 | if(!checkExists(indexDestination, shard)) {
85 | // Upload shard data
86 | String shardSource = snapshotWorkingLocation + "indices" + BaseESReducer.DIR_SEPARATOR + index + BaseESReducer.DIR_SEPARATOR + shard;
87 | transferDir(indexDestination, shardSource, new Integer(shard).toString());
88 | }
89 | }
90 | close();
91 | }
92 |
93 | /**
94 | * Rip out filesystem specific stuff off the path EG s3://
95 | * @param s
96 | * @return s
97 | */
98 | private String removeStorageSystemFromPath(String s) {
99 | for(STORAGE_SYSTEMS storageSystem : SnapshotTransportStrategy.STORAGE_SYSTEMS.values()) {
100 | s = s.replaceFirst(storageSystem.name() + "://", "");
101 | }
102 |
103 | return s;
104 | }
105 |
106 | /**
107 | * We've snapshotted an index with all data routed to a single shard (1 shard per reducer). Problem is
108 | * we don't know which shard # it routed all the data to. We can determine that by picking
109 | * out the largest shard folder and renaming it to the shard # we want it to be.
110 | */
111 | private String getShardSource(String index) throws IOException {
112 | // Get a list of shards in the snapshot
113 | String baseIndexLocation = snapshotWorkingLocation + "indices" + BaseESReducer.DIR_SEPARATOR + index + BaseESReducer.DIR_SEPARATOR;
114 | File file = new File(baseIndexLocation);
115 | String[] shardDirectories = file.list(directoryFilter);
116 |
117 | // Figure out which shard has all the data in it. Since we've routed all data to it, there'll only be one
118 | Long biggestDirLength = null;
119 | String biggestDir = null;
120 | for(String directory : shardDirectories) {
121 | File curDir = new File(baseIndexLocation + directory);
122 | long curDirLength = FileUtils.sizeOfDirectory(curDir);
123 | if(biggestDirLength == null || biggestDirLength < curDirLength) {
124 | biggestDir = directory;
125 | biggestDirLength = curDirLength;
126 | }
127 | }
128 |
129 | return biggestDir;
130 | }
131 |
132 | /**
133 | * We're building 1 shard at a time. Therefore each snapshot has a bunch of empty
134 | * shards and 1 shard with all the data in it. This deletes all the empty shard folders
135 | * for you.
136 | *
137 | * @param index
138 | * @param biggestDir
139 | * @throws IOException
140 | */
141 | private void cleanEmptyShards(String index, String biggestDir) throws IOException {
142 | String baseIndexLocation = snapshotWorkingLocation + "indices" + BaseESReducer.DIR_SEPARATOR + index + BaseESReducer.DIR_SEPARATOR;
143 | File file = new File(baseIndexLocation);
144 | String[] shardDirectories = file.list(directoryFilter);
145 |
146 | // Remove the empty shards
147 | for(String directory : shardDirectories) {
148 | if(!directory.equals(biggestDir)) {
149 | FileUtils.deleteDirectory(new File(baseIndexLocation + directory));
150 | }
151 | }
152 | }
153 |
154 | private class DirectoryFilter implements FilenameFilter {
155 |
156 | @Override
157 | public boolean accept(File current, String name) {
158 | return new File(current, name).isDirectory();
159 | }
160 | }
161 | }
162 |
--------------------------------------------------------------------------------
/src/main/java/com/inin/analytics/elasticsearch/transport/HDFSSnapshotTransport.java:
--------------------------------------------------------------------------------
1 | package com.inin.analytics.elasticsearch.transport;
2 |
3 | import java.io.File;
4 | import java.io.FileNotFoundException;
5 | import java.io.IOException;
6 |
7 | import org.apache.commons.lang.StringUtils;
8 | import org.apache.hadoop.conf.Configuration;
9 | import org.apache.hadoop.fs.FileSystem;
10 | import org.apache.hadoop.fs.Path;
11 | import org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException;
12 | import org.apache.hadoop.ipc.RemoteException;
13 | import org.slf4j.Logger;
14 | import org.slf4j.LoggerFactory;
15 |
16 | import com.inin.analytics.elasticsearch.BaseESReducer;
17 |
18 | public class HDFSSnapshotTransport extends BaseTransport {
19 | private FileSystem hdfsFileSystem;
20 | private static transient Logger logger = LoggerFactory.getLogger(HDFSSnapshotTransport.class);
21 |
22 | public HDFSSnapshotTransport(String snapshotWorkingLocation, String snapshotFinalDestination) {
23 | super(snapshotWorkingLocation, snapshotFinalDestination);
24 | }
25 |
26 | @Override
27 | protected void init() {
28 | Configuration conf = new Configuration();
29 | try {
30 | hdfsFileSystem = FileSystem.get(conf);
31 | } catch (IOException e) {
32 | throw new IllegalStateException("Unable to initialize HDFSSnapshotTransport because of ", e);
33 | }
34 | }
35 |
36 | @Override
37 | protected void close() {
38 |
39 | }
40 |
41 | private void ensurePathExists(String destination) throws IOException {
42 | String[] pieces = StringUtils.split(destination, BaseESReducer.DIR_SEPARATOR);
43 |
44 | String path = "";
45 | for(String piece : pieces) {
46 | if(StringUtils.isEmpty(piece)) {
47 | continue;
48 | }
49 | path = path + BaseESReducer.DIR_SEPARATOR + piece;
50 | if(!hdfsFileSystem.exists(new Path(path))) {
51 | try{
52 | hdfsFileSystem.mkdirs(new Path(path));
53 | } catch (IOException e) {
54 | logger.warn("Unable to create path " + path + " likely because it was created in another reducer thread.");
55 | }
56 | }
57 | }
58 | }
59 |
60 | @Override
61 | protected void transferFile(boolean deleteSource, String destination, String filename, String localDirectory) throws IOException {
62 | Path source = new Path(localDirectory + BaseESReducer.DIR_SEPARATOR + filename);
63 | ensurePathExists(destination);
64 |
65 | try{
66 | hdfsFileSystem.copyFromLocalFile(deleteSource, true, source, new Path(destination + BaseESReducer.DIR_SEPARATOR + filename));
67 | }
68 | catch(LeaseExpiredException | RemoteException e) {
69 | // This is an expected race condition where 2 reducers are trying to write the manifest files at the same time. That's okay, it only has to succeed once.
70 | logger.warn("Exception from 2 reducers writing the same file concurrently. One writer failed to obtain a lease. Destination " + destination + " filename " + filename + " localDirectory " + localDirectory, e);
71 | }
72 | }
73 |
74 | @Override
75 | protected void transferDir(String destination, String localShardPath, String shard) throws IOException {
76 | destination = destination + shard + BaseESReducer.DIR_SEPARATOR;
77 | ensurePathExists(destination);
78 | try{
79 | File[] files = new File(localShardPath).listFiles();
80 | for (File file : files) {
81 | transferFile(true, destination, file.getName(), localShardPath);
82 | }
83 | } catch(FileNotFoundException e) {
84 | throw new FileNotFoundException("Exception copying " + localShardPath + " to " + destination);
85 | }
86 | }
87 |
88 | @Override
89 | protected boolean checkExists(String destination, Integer shardNumber) throws IOException {
90 | return hdfsFileSystem.exists(new Path(destination + shardNumber));
91 | }
92 | }
93 |
--------------------------------------------------------------------------------
/src/main/java/com/inin/analytics/elasticsearch/transport/LocalFSSnapshotTransport.java:
--------------------------------------------------------------------------------
1 | package com.inin.analytics.elasticsearch.transport;
2 |
3 | import java.io.File;
4 | import java.io.IOException;
5 |
6 | import org.apache.commons.io.FileUtils;
7 |
8 | import com.google.common.base.Preconditions;
9 | import com.inin.analytics.elasticsearch.BaseESReducer;
10 |
11 | /**
12 | * Move the snapshot to locally connected storage
13 | *
14 | * @author drew
15 | *
16 | */
17 | public class LocalFSSnapshotTransport extends BaseTransport {
18 |
19 | public LocalFSSnapshotTransport(String snapshotWorkingLocation, String snapshotFinalDestination) {
20 | super(snapshotWorkingLocation, snapshotFinalDestination);
21 | }
22 |
23 | @Override
24 | protected void init() {
25 | // no-op
26 | }
27 |
28 | @Override
29 | protected void close() {
30 | // no-op
31 | }
32 |
33 | @Override
34 | protected void transferFile(boolean deleteSource, String destination, String filename, String localDirectory) throws IOException {
35 | File source = new File(localDirectory + BaseESReducer.DIR_SEPARATOR + filename);
36 | Preconditions.checkArgument(source.exists(), "Could not find source file: " + source.getAbsolutePath());
37 |
38 | File destinationDir = new File(destination);
39 | FileUtils.forceMkdir(destinationDir);
40 | FileUtils.copyFileToDirectory(source, destinationDir);
41 | if(deleteSource) {
42 | source.delete();
43 | }
44 | }
45 |
46 | @Override
47 | protected void transferDir(String destination, String source, String shard) throws IOException {
48 | File sourceDir = new File(source);
49 | Preconditions.checkArgument(sourceDir.exists(), "Could not find dir: " + source);
50 |
51 | File destinationDir = new File(destination + shard);
52 | FileUtils.forceMkdir(destinationDir);
53 | FileUtils.copyDirectory(sourceDir, destinationDir);
54 | }
55 |
56 | @Override
57 | protected boolean checkExists(String destination, Integer shardNumber) throws IOException {
58 | File destinationDir = new File(destination + shardNumber);
59 | return destinationDir.exists();
60 | }
61 |
62 | }
63 |
--------------------------------------------------------------------------------
/src/main/java/com/inin/analytics/elasticsearch/transport/S3SnapshotTransport.java:
--------------------------------------------------------------------------------
1 | package com.inin.analytics.elasticsearch.transport;
2 |
3 | import java.io.File;
4 | import java.io.FileInputStream;
5 | import java.io.FileNotFoundException;
6 | import java.io.IOException;
7 | import java.util.concurrent.Executors;
8 | import java.util.concurrent.ThreadFactory;
9 | import java.util.concurrent.ThreadPoolExecutor;
10 |
11 | import org.apache.commons.lang.StringUtils;
12 | import org.slf4j.Logger;
13 | import org.slf4j.LoggerFactory;
14 |
15 | import com.amazonaws.ClientConfiguration;
16 | import com.amazonaws.auth.DefaultAWSCredentialsProviderChain;
17 | import com.amazonaws.regions.Regions;
18 | import com.amazonaws.services.s3.AmazonS3Client;
19 | import com.amazonaws.services.s3.model.ListObjectsRequest;
20 | import com.amazonaws.services.s3.model.ObjectListing;
21 | import com.amazonaws.services.s3.model.ObjectMetadata;
22 | import com.amazonaws.services.s3.model.S3ObjectSummary;
23 | import com.amazonaws.services.s3.transfer.MultipleFileUpload;
24 | import com.amazonaws.services.s3.transfer.ObjectMetadataProvider;
25 | import com.amazonaws.services.s3.transfer.Transfer.TransferState;
26 | import com.amazonaws.services.s3.transfer.TransferManager;
27 | import com.amazonaws.services.s3.transfer.Upload;
28 | import com.google.common.base.Preconditions;
29 | import com.inin.analytics.elasticsearch.BaseESReducer;
30 |
31 | /**
32 | * Stitch together snapshots of ES shards as it pushes files to S3. If we could get the
33 | * snapshot gateway S3 plugin working we could potentially use that instead of using the aws sdk
34 | * directly, but there's some things holding that up.
35 | *
36 | * 1) That plugin needs some work to support MFA (at the time of writing it does not)
37 | * 2) We'll have to repackage ES with the plugin because installing plugins via the
38 | * embeded client isn't supported very well.
39 | * 3) This job is actually creating franken-snapshots. We're snapshotting each shard separately to reduce
40 | * the storage footprint per reducer task and then merging the snapshots together. To use the snapshot gateway
41 | * we would need a way to have the whole index on a single task tracker node. That means
42 | * beefy task trackers or mounting either some NFS or Ephemeral disks. Doing 1 shard at a time shrinks the problem.
43 | *
44 | * @author drew
45 | *
46 | */
47 | public class S3SnapshotTransport extends BaseTransport {
48 | private static transient Logger logger = LoggerFactory.getLogger(S3SnapshotTransport.class);
49 | private static final int S3_TRANSFER_THREAD_COUNT = 128;
50 | private TransferManager tx;
51 | private ObjectMetadataProvider objectMetadataProvider;
52 |
53 | /**
54 | * The default S3 thread pool in the aws sdk is 10 threads. ES Snapshots can be 100s of files, so parallelizing that
55 | * is advised.
56 | *
57 | * @return
58 | */
59 | public static ThreadPoolExecutor createDefaultExecutorService() {
60 | ThreadFactory threadFactory = new ThreadFactory() {
61 | private int threadCount = 1;
62 |
63 | public Thread newThread(Runnable r) {
64 | Thread thread = new Thread(r);
65 | thread.setName("s3-transfer-manager-worker-" + threadCount++);
66 | return thread;
67 | }
68 | };
69 | return (ThreadPoolExecutor)Executors.newFixedThreadPool(S3_TRANSFER_THREAD_COUNT, threadFactory);
70 | }
71 |
72 | public S3SnapshotTransport(String snapshotWorkingLocation, String snapshotFinalDestination) {
73 | super(snapshotWorkingLocation, snapshotFinalDestination);
74 | }
75 |
76 | public static AmazonS3Client getS3Client() {
77 | return (Regions.getCurrentRegion() != null) ?
78 | Regions.getCurrentRegion().createClient(AmazonS3Client.class,
79 | new DefaultAWSCredentialsProviderChain(),
80 | new ClientConfiguration()) :
81 | new AmazonS3Client();
82 | }
83 |
84 | @Override
85 | protected void init() {
86 | tx = new TransferManager(getS3Client(), createDefaultExecutorService());
87 |
88 | objectMetadataProvider = new ObjectMetadataProvider() {
89 | @Override
90 | public void provideObjectMetadata(File file, ObjectMetadata metadata) {
91 | metadata.setSSEAlgorithm("AES256");
92 | metadata.setContentLength(file.length());
93 | }
94 | };
95 | }
96 |
97 | @Override
98 | protected void close() {
99 | tx.shutdownNow();
100 | }
101 |
102 | protected void transferDir(String shardDestinationBucket, String localShardPath, String shard) {
103 | MultipleFileUpload mfu = tx.uploadDirectory(shardDestinationBucket + shard, null, new File(localShardPath), true, objectMetadataProvider);
104 |
105 | /**
106 | * TODO: Hadoop has a configurable timeout for how long a reducer can be non-responsive (usually 600s). If
107 | * this takes >600s hadoop will kill the task. We need to ping the reporter to let it know it's alive
108 | * in the case where the file transfer is taking a while.
109 | */
110 | while(!mfu.isDone()) {
111 | logger.info("Transfering to S3 completed %" + mfu.getProgress().getPercentTransferred());
112 | try {
113 | Thread.sleep(1000);
114 | } catch (InterruptedException e) {
115 | Thread.currentThread().interrupt();
116 | }
117 | }
118 | }
119 |
120 | protected void transferFile(boolean deleteSource, String bucket, String filename, String localDirectory) {
121 | File source = new File(localDirectory + BaseESReducer.DIR_SEPARATOR + filename);
122 | Preconditions.checkArgument(source.exists(), "Could not find source file: " + source.getAbsolutePath());
123 | logger.info("Transfering + " + source + " to " + bucket + " with key " + filename);
124 | FileInputStream fis;
125 | try {
126 | fis = new FileInputStream(source);
127 | ObjectMetadata objectMetadata = new ObjectMetadata();
128 | objectMetadata.setSSEAlgorithm("AES256");
129 | objectMetadata.setContentLength(source.length());
130 | Upload upload = tx.upload(bucket, filename, fis, objectMetadata);
131 |
132 | while(!upload.isDone());
133 | Preconditions.checkState(upload.getState().equals(TransferState.Completed), "File " + filename + " failed to upload with state: " + upload.getState());
134 | if(deleteSource) {
135 | source.delete();
136 | }
137 | } catch (FileNotFoundException e) {
138 | // Exception should never be thrown because the precondition above has already validated existence of file
139 | logger.error("Filename could not be found " + filename, e);
140 | }
141 | }
142 |
143 | @Override
144 | protected boolean checkExists(String destination, Integer shard) throws IOException {
145 | // Break that s3 path into bucket & key
146 | String[] pieces = StringUtils.split(destination, "/");
147 | String bucket = pieces[0];
148 | String key = destination.substring(bucket.length() + 1);
149 |
150 | // AWS SDK doesn't have an "exists" method so you have to list and check if the key is there. Thanks Obama
151 | ObjectListing objects = tx.getAmazonS3Client().listObjects(new ListObjectsRequest().withBucketName(bucket).withPrefix(key));
152 |
153 | for (S3ObjectSummary objectSummary: objects.getObjectSummaries()) {
154 | if (objectSummary.getKey().startsWith(key + shard)) {
155 | return true;
156 | }
157 | }
158 | return false;
159 | }
160 |
161 | }
162 |
--------------------------------------------------------------------------------
/src/main/java/com/inin/analytics/elasticsearch/transport/SnapshotTransportStrategy.java:
--------------------------------------------------------------------------------
1 | package com.inin.analytics.elasticsearch.transport;
2 |
3 | public class SnapshotTransportStrategy {
4 |
5 | public enum STORAGE_SYSTEMS {
6 | s3,
7 | hdfs
8 | }
9 |
10 | /**
11 | * Given a source & destination, return an appropriate transport implementation
12 | *
13 | * @param snapshotWorkingLocation
14 | * @param snapshotFinalDestination
15 | * @return BaseTransport
16 | */
17 | public static BaseTransport get(String snapshotWorkingLocation, String snapshotFinalDestination) {
18 | BaseTransport trasport = null;
19 | if(snapshotFinalDestination.startsWith(STORAGE_SYSTEMS.s3.name())) {
20 | trasport = new S3SnapshotTransport(snapshotWorkingLocation, snapshotFinalDestination);
21 | } else if(snapshotFinalDestination.startsWith(STORAGE_SYSTEMS.hdfs.name())) {
22 | trasport = new HDFSSnapshotTransport(snapshotWorkingLocation, snapshotFinalDestination);
23 | } else {
24 | trasport = new LocalFSSnapshotTransport(snapshotWorkingLocation, snapshotFinalDestination);
25 | }
26 | return trasport;
27 | }
28 | }
29 |
--------------------------------------------------------------------------------
/src/main/java/com/inin/analytics/elasticsearch/util/DateTimeDeserializer.java:
--------------------------------------------------------------------------------
1 | package com.inin.analytics.elasticsearch.util;
2 |
3 | import java.lang.reflect.Type;
4 |
5 | import org.joda.time.DateTime;
6 | import org.joda.time.DateTimeZone;
7 |
8 | import com.google.gson.JsonDeserializationContext;
9 | import com.google.gson.JsonDeserializer;
10 | import com.google.gson.JsonElement;
11 | import com.google.gson.JsonParseException;
12 | import com.google.gson.JsonPrimitive;
13 |
14 | public class DateTimeDeserializer implements JsonDeserializer {
15 | public DateTime deserialize(JsonElement json, Type typeOfT,
16 | JsonDeserializationContext context) throws JsonParseException {
17 | JsonPrimitive primitive = json.getAsJsonPrimitive();
18 | if(primitive.isNumber()) {
19 | return new DateTime(primitive.getAsLong()).withZone(DateTimeZone.UTC);
20 | } else {
21 | return new DateTime(primitive.getAsString()).withZone(DateTimeZone.UTC);
22 | }
23 | }
24 | }
--------------------------------------------------------------------------------
/src/main/java/com/inin/analytics/elasticsearch/util/DateTimeSerializer.java:
--------------------------------------------------------------------------------
1 | package com.inin.analytics.elasticsearch.util;
2 |
3 | import java.lang.reflect.Type;
4 |
5 | import org.joda.time.DateTime;
6 |
7 | import com.google.gson.JsonElement;
8 | import com.google.gson.JsonPrimitive;
9 | import com.google.gson.JsonSerializationContext;
10 | import com.google.gson.JsonSerializer;
11 |
12 | public class DateTimeSerializer implements JsonSerializer {
13 |
14 | @Override
15 | public JsonElement serialize(DateTime src, Type typeOfSrc,
16 | JsonSerializationContext context) {
17 | return new JsonPrimitive(src.toString());
18 | }
19 |
20 | }
21 |
--------------------------------------------------------------------------------
/src/main/java/com/inin/analytics/elasticsearch/util/GsonFactory.java:
--------------------------------------------------------------------------------
1 | package com.inin.analytics.elasticsearch.util;
2 |
3 | import org.joda.time.DateTime;
4 | import org.joda.time.LocalDate;
5 |
6 | import com.google.gson.Gson;
7 | import com.google.gson.GsonBuilder;
8 |
9 | public class GsonFactory {
10 | public static Gson buildGsonParser() {
11 | return buildGsonBuilder().create();
12 | }
13 |
14 | public static GsonBuilder buildGsonBuilder() {
15 | GsonBuilder builder = new GsonBuilder();
16 | builder.registerTypeAdapter(DateTime.class, new DateTimeSerializer());
17 | builder.registerTypeAdapter(DateTime.class, new DateTimeDeserializer());
18 | builder.registerTypeAdapter(LocalDate.class, new LocalDateSerializer());
19 | builder.registerTypeAdapter(LocalDate.class, new LocalDateDeserializer());
20 | return builder;
21 | }
22 | }
23 |
--------------------------------------------------------------------------------
/src/main/java/com/inin/analytics/elasticsearch/util/LocalDateDeserializer.java:
--------------------------------------------------------------------------------
1 | package com.inin.analytics.elasticsearch.util;
2 |
3 | import java.lang.reflect.Type;
4 |
5 |
6 | import org.joda.time.LocalDate;
7 |
8 | import com.google.gson.JsonDeserializationContext;
9 | import com.google.gson.JsonDeserializer;
10 | import com.google.gson.JsonElement;
11 | import com.google.gson.JsonParseException;
12 | import com.google.gson.JsonPrimitive;
13 |
14 | public class LocalDateDeserializer implements JsonDeserializer {
15 | public LocalDate deserialize(JsonElement json, Type typeOfT,
16 | JsonDeserializationContext context) throws JsonParseException {
17 | JsonPrimitive primitive = json.getAsJsonPrimitive();
18 | return new LocalDate(primitive.getAsString());
19 | }
20 | }
--------------------------------------------------------------------------------
/src/main/java/com/inin/analytics/elasticsearch/util/LocalDateSerializer.java:
--------------------------------------------------------------------------------
1 | package com.inin.analytics.elasticsearch.util;
2 |
3 | import java.lang.reflect.Type;
4 |
5 | import org.joda.time.LocalDate;
6 |
7 | import com.google.gson.JsonElement;
8 | import com.google.gson.JsonPrimitive;
9 | import com.google.gson.JsonSerializationContext;
10 | import com.google.gson.JsonSerializer;
11 |
12 | public class LocalDateSerializer implements JsonSerializer {
13 |
14 | @Override
15 | public JsonElement serialize(LocalDate src, Type typeOfSrc,
16 | JsonSerializationContext context) {
17 | return new JsonPrimitive(src.toString());
18 | }
19 |
20 | }
21 |
--------------------------------------------------------------------------------
/src/main/java/com/inin/analytics/elasticsearch/util/MurmurHash.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 | package com.inin.analytics.elasticsearch.util;
19 |
20 | public class MurmurHash {
21 | private static MurmurHash _instance = new MurmurHash();
22 |
23 | public static MurmurHash getInstance() {
24 | return _instance;
25 | }
26 |
27 | public int hash(byte[] data, int length, int seed) {
28 | int m = 0x5bd1e995;
29 | int r = 24;
30 |
31 | int h = seed ^ length;
32 |
33 | int len_4 = length >> 2;
34 |
35 | for (int i = 0; i < len_4; i++) {
36 | int i_4 = i << 2;
37 | int k = data[i_4 + 3];
38 | k = k << 8;
39 | k = k | (data[i_4 + 2] & 0xff);
40 | k = k << 8;
41 | k = k | (data[i_4 + 1] & 0xff);
42 | k = k << 8;
43 | k = k | (data[i_4 + 0] & 0xff);
44 | k *= m;
45 | k ^= k >>> r;
46 | k *= m;
47 | h *= m;
48 | h ^= k;
49 | }
50 |
51 | // avoid calculating modulo
52 | int len_m = len_4 << 2;
53 | int left = length - len_m;
54 |
55 | if (left != 0) {
56 | if (left >= 3) {
57 | h ^= (int) data[length - 3] << 16;
58 | }
59 | if (left >= 2) {
60 | h ^= (int) data[length - 2] << 8;
61 | }
62 | if (left >= 1) {
63 | h ^= (int) data[length - 1];
64 | }
65 |
66 | h *= m;
67 | }
68 |
69 | h ^= h >>> 13;
70 | h *= m;
71 | h ^= h >>> 15;
72 |
73 | return h;
74 | }
75 |
76 | /**
77 | * Calculate a hash using all bytes from the input argument,
78 | * and a provided seed value.
79 | * @param bytes input bytes
80 | * @param initval seed value
81 | * @return hash value
82 | */
83 | public int hash(byte[] bytes, int initval) {
84 | return hash(bytes, bytes.length, initval);
85 | }
86 | }
87 |
--------------------------------------------------------------------------------
/src/main/resources/hadoop-job.xml:
--------------------------------------------------------------------------------
1 |
2 | job
3 |
4 | jar
5 |
6 | false
7 |
8 |
9 | false
10 | runtime
11 | lib
12 |
13 | ${groupId}:${artifactId}
14 |
15 |
16 |
17 | true
18 |
19 | ${groupId}:${artifactId}
20 |
21 |
22 |
23 |
24 |
--------------------------------------------------------------------------------
/src/main/resources/logback.xml:
--------------------------------------------------------------------------------
1 |
2 |
3 |
4 |
5 |
6 |
7 |
8 |
9 |
10 |
11 |
12 |
13 |
14 |
15 |
16 |
17 |
18 | %d{HH:mm:ss.SSS} [%thread] %-5level %logger - %msg%n
19 |
20 |
21 |
22 |
23 |
24 |
25 |
26 |
27 |
--------------------------------------------------------------------------------
/src/test/java/com/inin/analytics/DateSerializerTest.java:
--------------------------------------------------------------------------------
1 | package com.inin.analytics;
2 |
3 | import static org.junit.Assert.*;
4 |
5 | import org.joda.time.LocalDate;
6 | import org.junit.Test;
7 |
8 | import com.google.gson.Gson;
9 | import com.inin.analytics.elasticsearch.index.rotation.ElasticSearchIndexMetadata;
10 | import com.inin.analytics.elasticsearch.index.routing.ElasticsearchRoutingStrategyV1;
11 | import com.inin.analytics.elasticsearch.util.GsonFactory;
12 |
13 | public class DateSerializerTest {
14 |
15 | @Test
16 | public void test() {
17 | ElasticSearchIndexMetadata indexMetadata = new ElasticSearchIndexMetadata();
18 | indexMetadata.setDate(new LocalDate("2015-02-09"));
19 | indexMetadata.setIndexNameAtBirth("test");
20 | indexMetadata.setNumShards(2);
21 | indexMetadata.setNumShardsPerOrg(3);
22 | indexMetadata.setRebuiltIndexAlias("alias");
23 | indexMetadata.setRoutingStrategyClassName(ElasticsearchRoutingStrategyV1.class.getName());
24 | Gson gson = GsonFactory.buildGsonBuilder().excludeFieldsWithoutExposeAnnotation().create();
25 | String json = gson.toJson(indexMetadata);
26 |
27 | assertEquals(json, "{\"indexNameAtBirth\":\"test\",\"rebuiltIndexAlias\":\"alias\",\"indexLocalDate\":\"2015-02-09\",\"numShards\":2,\"numShardsPerOrg\":3,\"routingStrategyClassName\":\"com.inin.analytics.elasticsearch.index.routing.ElasticsearchRoutingStrategyV1\"}");
28 | ElasticSearchIndexMetadata indexMetadataDeserialized = gson.fromJson(json, ElasticSearchIndexMetadata.class);
29 | assertEquals(indexMetadata.getDate(), indexMetadataDeserialized.getDate());
30 | assertEquals(indexMetadata.getIndexNameAtBirth(), indexMetadataDeserialized.getIndexNameAtBirth());
31 | assertEquals(indexMetadata.getNumShards(), indexMetadataDeserialized.getNumShards());
32 | assertEquals(indexMetadata.getNumShardsPerOrg(), indexMetadataDeserialized.getNumShardsPerOrg());
33 | assertEquals(indexMetadata.getRebuiltIndexAlias(), indexMetadataDeserialized.getRebuiltIndexAlias());
34 | assertEquals(indexMetadata.getRebuiltIndexName(), indexMetadataDeserialized.getRebuiltIndexName());
35 | assertEquals(indexMetadata.getRoutingStrategyClassName(), indexMetadataDeserialized.getRoutingStrategyClassName());
36 | }
37 |
38 | @Test
39 | public void testBackwardsCompatibility() {
40 | // We had a case where the indexDate was serialized wrong. This has since been fixed, but this test verifies that the indexDate is ignored and left null rather than blowing up on deserialize
41 | String oldJson = "\n" +
42 | "\n" +
43 | "{\n" +
44 | " \"indexNameAtBirth\": \"c141031\",\n" +
45 | " \"rebuiltIndexName\": \"c141031_build_399_20150206012509\",\n" +
46 | " \"rebuiltIndexAlias\": \"c141031r\",\n" +
47 | " \"indexDate\": {\n" +
48 | " \"iLocalMillis\": 1414713600000,\n" +
49 | " \"iChronology\": {\n" +
50 | " \"iBase\": {\n" +
51 | " \"iMinDaysInFirstWeek\": 4\n" +
52 | " }\n" +
53 | " }\n" +
54 | " },\n" +
55 | " \"numShards\": 5,\n" +
56 | " \"numShardsPerOrg\": 2,\n" +
57 | " \"routingStrategyClassName\": \"com.inin.analytics.elasticsearch.index.routing.ElasticsearchRoutingStrategyV1\"\n" +
58 | "}\n" +
59 | "\n" +
60 | "";
61 | Gson gson = GsonFactory.buildGsonBuilder().excludeFieldsWithoutExposeAnnotation().create();
62 | ElasticSearchIndexMetadata indexMetadataDeserialized = gson.fromJson(oldJson, ElasticSearchIndexMetadata.class);
63 | assertNull(indexMetadataDeserialized.getDate());
64 | assertEquals("c141031", indexMetadataDeserialized.getIndexNameAtBirth());
65 | assertEquals(5, indexMetadataDeserialized.getNumShards());
66 | assertEquals(2, indexMetadataDeserialized.getNumShardsPerOrg());
67 | assertEquals("c141031r", indexMetadataDeserialized.getRebuiltIndexAlias());
68 | assertEquals("c141031_build_399_20150206012509", indexMetadataDeserialized.getRebuiltIndexName());
69 | assertEquals("com.inin.analytics.elasticsearch.index.routing.ElasticsearchRoutingStrategyV1", indexMetadataDeserialized.getRoutingStrategyClassName());
70 | }
71 |
72 | @Test
73 | public void testMissingDate() {
74 | // We had a case where the indexDate was serialized wrong. This has since been fixed, but this test verifies that the indexDate is ignored and left null rather than blowing up on deserialize
75 | String oldJson = "\n" +
76 | "\n" +
77 | "{\n" +
78 | " \"indexNameAtBirth\": \"c141031\",\n" +
79 | " \"rebuiltIndexName\": \"c141031_build_399_20150206012509\",\n" +
80 | " \"rebuiltIndexAlias\": \"c141031r\",\n" +
81 | " \"numShards\": 5,\n" +
82 | " \"numShardsPerOrg\": 2,\n" +
83 | " \"routingStrategyClassName\": \"com.inin.analytics.elasticsearch.index.routing.ElasticsearchRoutingStrategyV1\"\n" +
84 | "}\n" +
85 | "\n" +
86 | "";
87 | Gson gson = GsonFactory.buildGsonBuilder().excludeFieldsWithoutExposeAnnotation().create();
88 | ElasticSearchIndexMetadata indexMetadataDeserialized = gson.fromJson(oldJson, ElasticSearchIndexMetadata.class);
89 | assertNull(indexMetadataDeserialized.getDate());
90 | assertEquals("c141031", indexMetadataDeserialized.getIndexNameAtBirth());
91 | assertEquals(5, indexMetadataDeserialized.getNumShards());
92 | assertEquals(2, indexMetadataDeserialized.getNumShardsPerOrg());
93 | assertEquals("c141031r", indexMetadataDeserialized.getRebuiltIndexAlias());
94 | assertEquals("c141031_build_399_20150206012509", indexMetadataDeserialized.getRebuiltIndexName());
95 | assertEquals("com.inin.analytics.elasticsearch.index.routing.ElasticsearchRoutingStrategyV1", indexMetadataDeserialized.getRoutingStrategyClassName());
96 | }
97 | }
98 |
--------------------------------------------------------------------------------
/src/test/java/com/inin/analytics/ElasticsearchRoutingStrategyV1Test.java:
--------------------------------------------------------------------------------
1 | package com.inin.analytics;
2 |
3 | import static org.junit.Assert.assertEquals;
4 | import static org.junit.Assert.assertTrue;
5 |
6 | import java.util.HashSet;
7 | import java.util.Set;
8 |
9 | import org.junit.Before;
10 | import org.junit.Test;
11 |
12 | import com.inin.analytics.elasticsearch.index.rotation.ElasticSearchIndexMetadata;
13 | import com.inin.analytics.elasticsearch.index.routing.ElasticsearchRoutingStrategyV1;
14 |
15 | public class ElasticsearchRoutingStrategyV1Test {
16 |
17 | private Set orgIds = new HashSet<>();
18 | private Set convIds = new HashSet<>();
19 |
20 | @Before
21 | public void setUp() throws Exception {
22 | // Note: this could be randomly generated, but if a test is to fail I want it to fail consistently
23 | orgIds.add("ed1121bf-5e61-4ac5-ad99-c24f8c4f79db");
24 | orgIds.add("b8864a7e-98d9-4bef-af1e-54c8bea7ae40");
25 | orgIds.add("decccc4f-2c96-4f4c-890f-eb1433ff4c90");
26 | orgIds.add("1650943b-b125-41cf-9729-3bd3e164da16");
27 | orgIds.add("005a22cc-afbb-4bbe-97e9-6f1447293ed7");
28 | orgIds.add("e29469e1-02a1-4d63-91ef-40affca740a8");
29 | orgIds.add("400cdb2f-7573-444e-9612-e218ff1c8387");
30 | orgIds.add("aec66b84-6c34-466b-8991-031cba01241b");
31 | orgIds.add("53adf13e-ce20-4112-9809-6aa29c60dfa5");
32 | orgIds.add("f7f8ff19-81bf-49b1-a896-e996674d5a1f");
33 | orgIds.add("2eb8db9f-d3ae-4d9a-ac78-55cb792e0d2d");
34 | orgIds.add("3b984743-49bd-47d9-b38f-da3f822db03a");
35 | orgIds.add("b68edfd1-305f-4d31-9443-605ba05eb5cc");
36 | orgIds.add("0c8ce21d-3bb5-4dab-9531-1e2f3320259e");
37 | orgIds.add("254f6bec-8b3d-48d2-976a-ba4a3517558b");
38 |
39 | convIds.add("0a3fe8fa-0291-4a28-87c7-2eeeda2295cd");
40 | convIds.add("38b261be-23c4-4fe6-846c-f06231ddf82f");
41 | convIds.add("3e4602bb-9c72-4174-b29f-b72dee356716");
42 | convIds.add("3ff398ac-b832-4085-9ba3-0d2756c03f21");
43 | convIds.add("8773bd12-3fdc-452f-b440-60bee40fadfc");
44 | convIds.add("a0f20cbe-19a4-4aff-833d-c0919d6cfe73");
45 | convIds.add("de48d484-23ce-4e0d-b465-de91b2f6ad72");
46 | convIds.add("be57d96e-7ee8-4bba-bc35-15e347b69bed");
47 | convIds.add("7cb1b182-b64a-4815-bc61-87714dbd0431");
48 | convIds.add("8b9bbfbc-34dc-45f4-8dee-d82a44fc9995");
49 | convIds.add("60ecef71-0a30-4798-aae7-63f6c1df0cf0");
50 | convIds.add("64d0431b-bb68-4045-8fff-5ae2ed4eed51");
51 | convIds.add("2e8df74f-3536-4044-aa13-1c1b273ab29f");
52 | }
53 |
54 | @Test
55 | public void testOrgOn7ShardsHashes() {
56 | ElasticSearchIndexMetadata indexMetadata = new ElasticSearchIndexMetadata();
57 | indexMetadata.setNumShards(10);
58 | indexMetadata.setNumShardsPerOrg(7);
59 |
60 | ElasticsearchRoutingStrategyV1 strategy = new ElasticsearchRoutingStrategyV1();
61 | strategy.configure(indexMetadata);
62 |
63 |
64 | for(String orgId : orgIds) {
65 | Set routingHashs = new HashSet<>();
66 | for(String convId : convIds) {
67 | String routingHash = strategy.getRoutingHash(orgId, convId);
68 | routingHashs.add(routingHash);
69 | }
70 | // Data was spread across #numShardsPerOrg shards
71 | assertEquals(routingHashs.size(), 7);
72 |
73 | // Possible hashes contain the routing hashes
74 | String[] possibleHashes = strategy.getPossibleRoutingHashes(orgId);
75 | assertEquals(possibleHashes.length, 7);
76 | for(String possibleHash : possibleHashes) {
77 | assertTrue(routingHashs.contains(possibleHash));
78 | }
79 | }
80 | }
81 |
82 | @Test
83 | public void testOrgOn1ShardsHashes() {
84 | ElasticSearchIndexMetadata indexMetadata = new ElasticSearchIndexMetadata();
85 | indexMetadata.setNumShards(5);
86 | indexMetadata.setNumShardsPerOrg(1);
87 |
88 | ElasticsearchRoutingStrategyV1 strategy = new ElasticsearchRoutingStrategyV1();
89 | strategy.configure(indexMetadata);
90 | for(String orgId : orgIds) {
91 | Set routingHashs = new HashSet<>();
92 | for(String convId : convIds) {
93 | String routingHash = strategy.getRoutingHash(orgId, convId);
94 | routingHashs.add(routingHash);
95 | }
96 | // Data was spread across #numShardsPerOrg shards
97 | assertEquals(routingHashs.size(), 1);
98 |
99 | // Possible hashes contain the routing hashes
100 | String[] possibleHashes = strategy.getPossibleRoutingHashes(orgId);
101 | assertEquals(possibleHashes.length, 1);
102 | for(String possibleHash : possibleHashes) {
103 | assertTrue(routingHashs.contains(possibleHash));
104 | }
105 | }
106 | }
107 |
108 | @Test
109 | public void testSingleShardIndex() {
110 | ElasticSearchIndexMetadata indexMetadata = new ElasticSearchIndexMetadata();
111 | indexMetadata.setNumShards(1);
112 | indexMetadata.setNumShardsPerOrg(1);
113 |
114 | ElasticsearchRoutingStrategyV1 strategy = new ElasticsearchRoutingStrategyV1();
115 | strategy.configure(indexMetadata);
116 | Set routingHashs = new HashSet<>();
117 | for(String orgId : orgIds) {
118 | for(String convId : convIds) {
119 | String routingHash = strategy.getRoutingHash(orgId, convId);
120 | routingHashs.add(routingHash);
121 | }
122 | }
123 | assertEquals(routingHashs.size(), 1);
124 | }
125 |
126 | @Test
127 | public void testOrgDistribution() {
128 | ElasticSearchIndexMetadata indexMetadata = new ElasticSearchIndexMetadata();
129 | indexMetadata.setNumShards(5);
130 | indexMetadata.setNumShardsPerOrg(1);
131 | ElasticsearchRoutingStrategyV1 strategy = new ElasticsearchRoutingStrategyV1();
132 | strategy.configure(indexMetadata);
133 |
134 | Set routingHashs = new HashSet<>();
135 | for(String orgId : orgIds) {
136 | routingHashs.add(strategy.getRoutingHash(orgId, "713729b0-91d1-4006-9317-8db4bc113be4"));
137 | }
138 | assertEquals(routingHashs.size(), indexMetadata.getNumShards());
139 | }
140 |
141 | }
142 |
--------------------------------------------------------------------------------
/src/test/java/com/inin/analytics/IndexRotationStrategyZookeeperTest.java:
--------------------------------------------------------------------------------
1 | package com.inin.analytics;
2 |
3 | import static org.junit.Assert.assertEquals;
4 | import static org.junit.Assert.fail;
5 |
6 | import java.io.IOException;
7 |
8 | import org.apache.curator.framework.CuratorFramework;
9 | import org.apache.curator.framework.CuratorFrameworkFactory;
10 | import org.apache.curator.retry.ExponentialBackoffRetry;
11 | import org.apache.curator.test.TestingServer;
12 | import org.joda.time.DateTime;
13 | import org.junit.After;
14 | import org.junit.Before;
15 | import org.junit.Test;
16 | import org.junit.runner.RunWith;
17 | import org.mockito.runners.MockitoJUnitRunner;
18 |
19 | import com.inin.analytics.elasticsearch.index.rotation.ExampleElasticsearchIndexRotationStrategyZookeeper;
20 | import com.inin.analytics.elasticsearch.index.rotation.RebuildPipelineState;
21 | import com.inin.analytics.elasticsearch.index.rotation.ElasticSearchIndexMetadata;
22 | import com.inin.analytics.elasticsearch.index.routing.ElasticsearchRoutingStrategyV1;
23 | import com.inin.analytics.elasticsearch.index.selector.RealtimeIndexSelectionStrategyLagged;
24 |
25 |
26 | @RunWith(MockitoJUnitRunner.class)
27 | public class IndexRotationStrategyZookeeperTest {
28 |
29 | private static final Integer ZOOKEEPER_EMBEDED_PORT = 2181;
30 | private TestingServer zk = null;
31 | private ExampleElasticsearchIndexRotationStrategyZookeeper rotation;
32 |
33 | @Before
34 | public void setup() throws Exception {
35 | zk = new TestingServer(ZOOKEEPER_EMBEDED_PORT);
36 | rotation = new ExampleElasticsearchIndexRotationStrategyZookeeper();
37 | rotation.setCurator(getCurator());
38 | rotation.init();
39 | }
40 |
41 | public CuratorFramework getCurator() throws InterruptedException {
42 | ExponentialBackoffRetry retryPolicy = new ExponentialBackoffRetry(1000, 3);
43 | CuratorFramework curator = CuratorFrameworkFactory.newClient("localhost", 10000, 10000, retryPolicy);
44 | curator.start();
45 | curator.getZookeeperClient().blockUntilConnectedOrTimedOut();
46 | return curator;
47 | }
48 |
49 | @After
50 | public void close() throws IOException {
51 | zk.close();
52 | }
53 |
54 | @Test
55 | public final void testRegisterAndGet() throws Exception {
56 | DateTime now = new DateTime();
57 |
58 | ElasticSearchIndexMetadata metaData = new ElasticSearchIndexMetadata();
59 | metaData.setIndexNameAtBirth("a");
60 | metaData.setNumShardsPerOrg(2);
61 | metaData.setRebuiltIndexAlias("b");
62 | metaData.setRebuiltIndexName("b");
63 |
64 | metaData.setRoutingStrategyClassName(ElasticsearchRoutingStrategyV1.class.getName());
65 | rotation.registerIndexAvailableOnRotation(metaData);
66 |
67 | // NodeCache curator recipe is async so we loop for up to 1sec waiting for the watcher to react. Polling sucks, but it beats Thread.sleep(1000) and generally happens in a few ms.
68 | long timer = System.currentTimeMillis();
69 | while(true) {
70 | ElasticSearchIndexMetadata readMetaData = rotation.getElasticSearchIndexMetadata(metaData.getIndexNameAtBirth());
71 | assertEquals(readMetaData.getIndexNameAtBirth(), metaData.getIndexNameAtBirth());
72 |
73 |
74 | if(readMetaData.getRebuiltIndexName() != null) {
75 | // Assert that rotation lag is accounted for
76 | assertEquals(readMetaData.getRebuiltIndexName(), metaData.getRebuiltIndexName());
77 |
78 | break;
79 | }
80 | if(System.currentTimeMillis() - timer > 1000) {
81 | fail("NodeCache failed to update with latest value in a reasonable amount of time");
82 | }
83 | }
84 | }
85 |
86 | @Test
87 | public void testRealtimeIndexSelectionStrategyLagged() {
88 | DateTime now = new DateTime();
89 | ElasticSearchIndexMetadata metaData = new ElasticSearchIndexMetadata();
90 | metaData.setIndexNameAtBirth("a");
91 | metaData.setNumShardsPerOrg(2);
92 | metaData.setRebuiltIndexAlias("b");
93 | metaData.setRebuiltIndexName("bb");
94 | metaData.setRoutingStrategyClassName(ElasticsearchRoutingStrategyV1.class.getName());
95 |
96 | RealtimeIndexSelectionStrategyLagged strategy = new RealtimeIndexSelectionStrategyLagged(2);
97 | metaData.setDate(now.toLocalDate());
98 | assertEquals(strategy.getIndexReadable(metaData), "a");
99 | assertEquals(strategy.getIndexWritable(metaData), "a");
100 |
101 | metaData.setDate(now.minusDays(1).toLocalDate());
102 | assertEquals(strategy.getIndexReadable(metaData), "a");
103 | assertEquals(strategy.getIndexWritable(metaData), "a");
104 |
105 | metaData.setDate(now.minusDays(2).toLocalDate());
106 | assertEquals(strategy.getIndexReadable(metaData), "b");
107 | assertEquals(strategy.getIndexWritable(metaData), "bb");
108 |
109 | metaData.setDate(now.minusDays(3).toLocalDate());
110 | assertEquals(strategy.getIndexReadable(metaData), "b");
111 | assertEquals(strategy.getIndexWritable(metaData), "bb");
112 |
113 | metaData.setRebuiltIndexName(null);
114 | metaData.setRebuiltIndexAlias(null);
115 | assertEquals(strategy.getIndexWritable(metaData), "a");
116 | }
117 |
118 | @Test
119 | public void testRealtimeIndexSelectionStrategyLagged_BackwardsCompatibility() {
120 | DateTime now = new DateTime();
121 | ElasticSearchIndexMetadata metaData = new ElasticSearchIndexMetadata();
122 | metaData.setIndexNameAtBirth("a");
123 | metaData.setNumShardsPerOrg(2);
124 | metaData.setRebuiltIndexAlias("b");
125 | metaData.setRoutingStrategyClassName(ElasticsearchRoutingStrategyV1.class.getName());
126 |
127 | RealtimeIndexSelectionStrategyLagged strategy = new RealtimeIndexSelectionStrategyLagged(2);
128 | metaData.setDate(now.minusDays(2).toLocalDate());
129 | assertEquals(strategy.getIndexReadable(metaData), "b");
130 | assertEquals(strategy.getIndexWritable(metaData), "b");
131 | }
132 |
133 | @Test
134 | public void testRebuildPipelineStateRunning() {
135 | rotation.updateRebuildPipelineState(RebuildPipelineState.RUNNING);
136 | assertEquals(rotation.getRebuildPipelineState(), RebuildPipelineState.RUNNING);
137 | }
138 |
139 | @Test
140 | public void testRebuildPipelineStateComplete() {
141 | rotation.updateRebuildPipelineState(RebuildPipelineState.COMPLETE);
142 | assertEquals(rotation.getRebuildPipelineState(), RebuildPipelineState.COMPLETE);
143 | }
144 |
145 | }
146 |
--------------------------------------------------------------------------------