diff --git a/common/src/main/java/io/druid/common/config/Log4jShutdown.java b/common/src/main/java/io/druid/common/config/Log4jShutdown.java new file mode 100644 index 000000000000..9353ae3af667 --- /dev/null +++ b/common/src/main/java/io/druid/common/config/Log4jShutdown.java @@ -0,0 +1,141 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.common.config; + +import org.apache.logging.log4j.core.util.Cancellable; +import org.apache.logging.log4j.core.util.ShutdownCallbackRegistry; + +import java.util.Queue; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; + +public class Log4jShutdown implements ShutdownCallbackRegistry, org.apache.logging.log4j.core.LifeCycle +{ + private final AtomicReference state = new AtomicReference<>(State.INITIALIZED); + private final Queue shutdownCallbacks = new ConcurrentLinkedQueue<>(); + private final AtomicBoolean callbacksRun = new AtomicBoolean(false); + + @Override + public Cancellable addShutdownCallback(final Runnable callback) + { + if (callback == null) { + throw new NullPointerException("callback"); + } + if (!isStarted()) { + throw new IllegalStateException("Not started"); + } + final Cancellable cancellable = new Cancellable() + { + private volatile boolean cancelled = false; + private final AtomicBoolean ran = new AtomicBoolean(false); + + @Override + public void cancel() + { + cancelled = true; + } + + @Override + public void run() + { + if (!cancelled) { + if (ran.compareAndSet(false, true)) { + callback.run(); + } + } + } + }; + shutdownCallbacks.add(cancellable); + if (!isStarted()) { + // We are shutting down in the middle of registering... Make sure the callback fires + callback.run(); + throw new IllegalStateException("Shutting down while adding shutdown hook. Callback fired just in case"); + } + return cancellable; + } + + @Override + public State getState() + { + return state.get(); + } + + @Override + public void initialize() + { + // NOOP, state is always at least INITIALIZED + } + + @Override + public void start() + { + if (!state.compareAndSet(State.INITIALIZED, State.STARTED)) { // Skip STARTING + throw new IllegalStateException(String.format("Expected state [%s] found [%s]", State.INITIALIZED, state.get())); + } + } + + @Override + public void stop() + { + if (callbacksRun.get()) { + return; + } + if (!state.compareAndSet(State.STARTED, State.STOPPED)) { + throw new IllegalStateException(String.format("Expected state [%s] found [%s]", State.STARTED, state.get())); + } + } + + public void runCallbacks() + { + if (!callbacksRun.compareAndSet(false, true)) { + // Already run, skip + return; + } + stop(); + RuntimeException e = null; + for (Runnable callback = shutdownCallbacks.poll(); callback != null; callback = shutdownCallbacks.poll()) { + try { + callback.run(); + } + catch (RuntimeException ex) { + if (e == null) { + e = new RuntimeException("Error running callback"); + } + e.addSuppressed(ex); + } + } + if (e != null) { + throw e; + } + } + + @Override + public boolean isStarted() + { + return State.STARTED.equals(getState()); + } + + @Override + public boolean isStopped() + { + return State.STOPPED.equals(getState()); + } +} diff --git a/docs/content/configuration/coordinator.md b/docs/content/configuration/coordinator.md index ad1e0d79d7fb..0fd618bc55dd 100644 --- a/docs/content/configuration/coordinator.md +++ b/docs/content/configuration/coordinator.md @@ -34,7 +34,7 @@ The coordinator node uses several of the global configs in [Configuration](../co |Property|Description|Default| |--------|-----------|-------| |`druid.manager.config.pollDuration`|How often the manager polls the config table for updates.|PT1m| -|`druid.manager.segment.pollDuration`|The duration between polls the Coordinator does for updates to the set of active segments. Generally defines the amount of lag time it can take for the coordinator to notice new segments.|PT1M| +|`druid.manager.segments.pollDuration`|The duration between polls the Coordinator does for updates to the set of active segments. Generally defines the amount of lag time it can take for the coordinator to notice new segments.|PT1M| |`druid.manager.rules.pollDuration`|The duration between polls the Coordinator does for updates to the set of active rules. Generally defines the amount of lag time it can take for the coordinator to notice rules.|PT1M| |`druid.manager.rules.defaultTier`|The default tier from which default rules will be loaded from.|_default| |`druid.manager.rules.alertThreshold`|The duration after a failed poll upon which an alert should be emitted.|PT10M| diff --git a/docs/content/configuration/index.md b/docs/content/configuration/index.md index 708755e59311..06f7e2ac77c1 100644 --- a/docs/content/configuration/index.md +++ b/docs/content/configuration/index.md @@ -118,6 +118,7 @@ The following monitors are available: |`io.druid.server.metrics.HistoricalMetricsMonitor`|Reports statistics on Historical nodes.| |`com.metamx.metrics.JvmMonitor`|Reports JVM-related statistics.| |`io.druid.segment.realtime.RealtimeMetricsMonitor`|Reports statistics on Realtime nodes.| +|`io.druid.server.metrics.EventReceiverFirehoseMonitor`|Reports how many events have been queued in the EventReceiverFirehose.| ### Emitting Metrics diff --git a/docs/content/configuration/indexing-service.md b/docs/content/configuration/indexing-service.md index 68d6612f41de..275b6c6db12f 100644 --- a/docs/content/configuration/indexing-service.md +++ b/docs/content/configuration/indexing-service.md @@ -251,6 +251,7 @@ Middle managers pass their configurations down to their child peons. The middle |`druid.indexer.runner.javaOpts`|-X Java options to run the peon in its own JVM.|""| |`druid.indexer.runner.maxZnodeBytes`|The maximum size Znode in bytes that can be created in Zookeeper.|524288| |`druid.indexer.runner.startPort`|The port that peons begin running on.|8100| +|`druid.indexer.runner.separateIngestionEndpoint`|Use separate server and consequently separate jetty thread pool for ingesting events|false| |`druid.worker.ip`|The IP of the worker.|localhost| |`druid.worker.version`|Version identifier for the middle manager.|0| |`druid.worker.capacity`|Maximum number of tasks the middle manager can accept.|Number of available processors - 1| @@ -272,6 +273,15 @@ Additional peon configs include: |`druid.indexer.task.hadoopWorkingPath`|Temporary working directory for Hadoop tasks.|/tmp/druid-indexing| |`druid.indexer.task.defaultRowFlushBoundary`|Highest row count before persisting to disk. Used for indexing generating tasks.|50000| |`druid.indexer.task.defaultHadoopCoordinates`|Hadoop version to use with HadoopIndexTasks that do not request a particular version.|org.apache.hadoop:hadoop-client:2.3.0| +|`druid.indexer.task.gracefulShutdownTimeout`|Wait this long on middleManager restart for restorable tasks to gracefully exit.|PT5M| +|`druid.indexer.task.directoryLockTimeout`|Wait this long for zombie peons to exit before giving up on their replacements.|PT10M| + +If `druid.indexer.runner.separateIngestionEndpoint` is set to true then following configurations are available for the ingestion server at peon: + +|Property|Description|Default| +|--------|-----------|-------| +|`druid.indexer.server.chathandler.http.numThreads`|Number of threads for HTTP requests.|Math.max(10, (Number of available processors * 17) / 16 + 2) + 30| +|`druid.indexer.server.chathandler.http.maxIdleTime`|The Jetty max idle time for a connection.|PT5m| If the peon is running in remote mode, there must be an overlord up and running. Peons in remote mode can set the following configurations: diff --git a/docs/content/development/datasketches-aggregators.md b/docs/content/development/datasketches-aggregators.md new file mode 100644 index 000000000000..62c70ae6cc74 --- /dev/null +++ b/docs/content/development/datasketches-aggregators.md @@ -0,0 +1,141 @@ +--- +layout: doc_page +--- + +## DataSketches aggregator +Druid aggregators based on [datasketches]()http://datasketches.github.io/) library. Note that sketch algorithms are approxiate, see details in the "Accuracy" section of the datasketches doc. +At ingestion time, this aggregator creates the theta sketch objects which get stored in Druid segments. Logically speaking, a theta sketch object can be thought of as a Set data structure. At query time, sketches are read and aggregated(set unioned) together. In the end, by default, you receive the estimate of number of unique entries in the sketch object. Also, You can use post aggregators to do union, intersection or difference on sketch columns in the same row. +Note that you can use `thetaSketch` aggregator on columns which were not ingested using same, it will return estimated cardinality of the column. It is recommended to use it at ingestion time as well to make querying faster. + +### Aggregators + +```json +{ + "type" : "thetaSketch", + "name" : , + "fieldName" : , + + //following boolean field is optional. This should only be used at + //indexing time if your input data contains theta sketch objects. + //that would be the case if you use datasketches library outside of Druid, + //say with Pig/Hive, to produce the data that you are ingesting into Druid + "isInputThetaSketch": false + + //following field is optional, default = 16384. must be a power of 2. + //Internally, size refers to the maximum number + //of entries sketch object will retain, higher size would mean higher + //accuracy but higher space needed to store those sketches. + //note that after you index with a particular size, druid will persist sketch in segments + //and you will use size greater or equal to that at query time. + //See [theta-size](http://datasketches.github.io/docs/ThetaSize.html) for details. + //In general, We recommend just sticking to default size, which has worked well. + "size": 16384 + } +``` + +### Post Aggregators + +#### Sketch Estimator +```json +{ "type" : "thetaSketchEstimate", "name": , "fieldName" : } +``` + +#### Sketch Operations +```json +{ "type" : "thetaSketchSetOp", "name": , "func": , "fields" : } +``` + +### Examples + +Assuming, you have a dataset containing (timestamp, product, user_id). You want to answer questions like + +How many unique users visited product A? +How many unique users visited both product A and product B? + +to answer above questions, you would index your data using following aggregator. + +```json +{ "type": "thetaSketch", "name": "user_id_sketch", "fieldName": "user_id" } +``` + +then, sample query for, How many unique users visited product A? +```json +{ + "queryType": "groupBy", + "dataSource": "test_datasource", + "granularity": "ALL", + "dimensions": [], + "aggregations": [ + { "type": "thetaSketch", "name": "unique_users", "fieldName": "user_id_sketch" } + ], + "filter": { "type": "selector", "dimension": "product", "value": "A" }, + "intervals": [ "2014-10-19T00:00:00.000Z/2014-10-22T00:00:00.000Z" ] +} +``` + +sample query for, How many unique users visited both product A and B? + +```json +{ + "queryType": "groupBy", + "dataSource": "test_datasource", + "granularity": "ALL", + "dimensions": [], + "filter": { + "type": "or", + "fields": [ + {"type": "selector", "dimension": "product", "value": "A"}, + {"type": "selector", "dimension": "product", "value": "B"} + ] + }, + "aggregations": [ + { + "type" : "filtered", + "filter" : { + "type" : "selector", + "dimension" : "product", + "value" : "A" + }, + "aggregator" : { + "type": "thetaSketch", "name": "A_unique_users", "fieldName": "user_id_sketch" + } + }, + { + "type" : "filtered", + "filter" : { + "type" : "selector", + "dimension" : "product", + "value" : "B" + }, + "aggregator" : { + "type": "thetaSketch", "name": "B_unique_users", "fieldName": "user_id_sketch" + } + } + ], + "postAggregations": [ + { + "type": "thetaSketchEstimate", + "name": "final_unique_users", + "field": + { + "type": "thetaSketchSetOp", + "name": "final_unique_users_sketch", + "func": "INTERSECT", + "fields": [ + { + "type": "fieldAccess", + "fieldName": "A_unique_users" + }, + { + "type": "fieldAccess", + "fieldName": "B_unique_users" + } + ] + } + } + ], + "intervals": [ + "2014-10-19T00:00:00.000Z/2014-10-22T00:00:00.000Z" + ] +} +``` diff --git a/docs/content/ingestion/realtime-ingestion.md b/docs/content/ingestion/realtime-ingestion.md index 394e1a756f76..1a13f885ce81 100644 --- a/docs/content/ingestion/realtime-ingestion.md +++ b/docs/content/ingestion/realtime-ingestion.md @@ -134,7 +134,7 @@ The tuningConfig is optional and default parameters will be used if no tuningCon |Field|Type|Description|Required| |-----|----|-----------|--------| |type|String|This should always be 'realtime'.|no| -|maxRowsInMemory|Integer|The number of rows to aggregate before persisting. This number is the post-aggregation rows, so it is not equivalent to the number of input events, but the number of aggregated rows that those events result in. This is used to manage the required JVM heap size. Maximum heap memory usage for indexing scales with maxRowsInMemory * (2 + maxPendingPersists).|no (default == 5 million)| +|maxRowsInMemory|Integer|The number of rows to aggregate before persisting. This number is the post-aggregation rows, so it is not equivalent to the number of input events, but the number of aggregated rows that those events result in. This is used to manage the required JVM heap size. Maximum heap memory usage for indexing scales with maxRowsInMemory * (2 + maxPendingPersists).|no (default == 500000)| |windowPeriod|ISO 8601 Period String|The amount of lag time to allow events. This is configured with a 10 minute window, meaning that any event more than 10 minutes ago will be thrown away and not included in the segment generated by the realtime server.|no (default == PT10m)| |intermediatePersistPeriod|ISO8601 Period String|The period that determines the rate at which intermediate persists occur. These persists determine how often commits happen against the incoming realtime stream. If the realtime data loading process is interrupted at time T, it should be restarted to re-read data that arrived at T minus this period.|no (default == PT10m)| |basePersistDirectory|String|The directory to put things that need persistence. The plumber is responsible for the actual intermediate persists and this tells it where to store those persists.|no (default == java tmp dir)| diff --git a/docs/content/operations/metrics.md b/docs/content/operations/metrics.md index 0a445a29655f..68b57a8ab669 100644 --- a/docs/content/operations/metrics.md +++ b/docs/content/operations/metrics.md @@ -156,6 +156,14 @@ These metrics are only available if the JVMMonitor module is included. |`jvm/gc/count`|Garbage collection count.|gcName.|< 100| |`jvm/gc/time`|Garbage collection time.|gcName.|< 1s| +### EventReceiverFirehose + +The following metric is only available if the EventReceiverFirehoseMonitor module is included. + +|Metric|Description|Dimensions|Normal Value| +|------|-----------|----------|------------| +|`ingest/events/buffered`|Number of events queued in the EventReceiverFirehose's buffer|serviceName, bufferCapacity.|Equal to current # of events in the buffer queue.| + ## Sys These metrics are only available if the SysMonitor module is included. diff --git a/docs/content/querying/granularities.md b/docs/content/querying/granularities.md index 02833f402d3a..2b4930808f27 100644 --- a/docs/content/querying/granularities.md +++ b/docs/content/querying/granularities.md @@ -11,11 +11,171 @@ It can be specified either as a string for simple granularities or as an object Simple granularities are specified as a string and bucket timestamps by their UTC time (e.g., days start at 00:00 UTC). -Supported granularity strings are: `all`, `none`, `minute`, `fifteen_minute`, `thirty_minute`, `hour` and `day` +Supported granularity strings are: `all`, `none`, `minute`, `fifteen_minute`, `thirty_minute`, `hour` and `day`. * `all` buckets everything into a single bucket * `none` does not bucket data (it actually uses the granularity of the index - minimum here is `none` which means millisecond granularity). Using `none` in a [TimeseriesQuery](../querying/timeseriesquery.html) is currently not recommended (the system will try to generate 0 values for all milliseconds that didn’t exist, which is often a lot). +#### Example: + +Suppose you have data below stored in Druid with millisecond ingestion granularity, + +``` json +{"timestamp": "2013-08-31T01:02:33Z", "page": "AAA", "language" : "en"} +{"timestamp": "2013-09-01T01:02:33Z", "page": "BBB", "language" : "en"} +{"timestamp": "2013-09-02T23:32:45Z", "page": "CCC", "language" : "en"} +{"timestamp": "2013-09-03T03:32:45Z", "page": "DDD", "language" : "en"} +``` + +After submitting a groupBy query with `hour` granularity, + +``` json +{ + "queryType":"groupBy", + "dataSource":"my_dataSource", + "granularity":"hour", + "dimensions":[ + "language" + ], + "aggregations":[ + { + "type":"count", + "name":"count" + } + ], + "intervals":[ + "2000-01-01T00:00Z/3000-01-01T00:00Z" + ] +} +``` + +you will get + +``` json +[ { + "version" : "v1", + "timestamp" : "2013-08-31T01:00:00.000Z", + "event" : { + "count" : 1, + "language" : "en" + } +}, { + "version" : "v1", + "timestamp" : "2013-09-01T01:00:00.000Z", + "event" : { + "count" : 1, + "language" : "en" + } +}, { + "version" : "v1", + "timestamp" : "2013-09-02T23:00:00.000Z", + "event" : { + "count" : 1, + "language" : "en" + } +}, { + "version" : "v1", + "timestamp" : "2013-09-03T03:00:00.000Z", + "event" : { + "count" : 1, + "language" : "en" + } +} ] +``` + +Note that all the empty buckets are discarded. + + +If you change the granularity to `day`, you will get + +``` json +[ { + "version" : "v1", + "timestamp" : "2013-08-31T00:00:00.000Z", + "event" : { + "count" : 1, + "language" : "en" + } +}, { + "version" : "v1", + "timestamp" : "2013-09-01T00:00:00.000Z", + "event" : { + "count" : 1, + "language" : "en" + } +}, { + "version" : "v1", + "timestamp" : "2013-09-02T00:00:00.000Z", + "event" : { + "count" : 1, + "language" : "en" + } +}, { + "version" : "v1", + "timestamp" : "2013-09-03T00:00:00.000Z", + "event" : { + "count" : 1, + "language" : "en" + } +} ] +``` + + +If you change the granularity to `none`, you will get the same results as setting it to the ingestion granularity. + +``` json +[ { + "version" : "v1", + "timestamp" : "2013-08-31T01:02:33.000Z", + "event" : { + "count" : 1, + "language" : "en" + } +}, { + "version" : "v1", + "timestamp" : "2013-09-01T01:02:33.000Z", + "event" : { + "count" : 1, + "language" : "en" + } +}, { + "version" : "v1", + "timestamp" : "2013-09-02T23:32:45.000Z", + "event" : { + "count" : 1, + "language" : "en" + } +}, { + "version" : "v1", + "timestamp" : "2013-09-03T03:32:45.000Z", + "event" : { + "count" : 1, + "language" : "en" + } +} ] +``` + +Having a query granularity smaller than the ingestion granularity doesn't make sense, +because information about that smaller granularity is not present in the indexed data. +So, if the query granularity is smaller than the ingestion granularity, druid produces +results that are equivalent to having set the query granularity to the ingestion granularity. +See `queryGranularity` in [Ingestion Spec](../ingestion/index.html). + + +If you change the granularity to `all`, you will get everything aggregated in 1 bucket, + +``` json +[ { + "version" : "v1", + "timestamp" : "2000-01-01T00:00:00.000Z", + "event" : { + "count" : 4, + "language" : "en" + } +} ] +``` + + ### Duration Granularities Duration granularities are specified as an exact duration in milliseconds and timestamps are returned as UTC. Duration granularity values are in millis. @@ -34,6 +194,106 @@ This chunks up every 2 hours. This chunks up every hour on the half-hour. +#### Example: + +Reusing the data in the previous example, after submitting a groupBy query with 24 hours duration, + +``` json +{ + "queryType":"groupBy", + "dataSource":"my_dataSource", + "granularity":{"type": "duration", "duration": "86400000"}, + "dimensions":[ + "language" + ], + "aggregations":[ + { + "type":"count", + "name":"count" + } + ], + "intervals":[ + "2000-01-01T00:00Z/3000-01-01T00:00Z" + ] +} +``` + +you will get + +``` json +[ { + "version" : "v1", + "timestamp" : "2013-08-31T00:00:00.000Z", + "event" : { + "count" : 1, + "language" : "en" + } +}, { + "version" : "v1", + "timestamp" : "2013-09-01T00:00:00.000Z", + "event" : { + "count" : 1, + "language" : "en" + } +}, { + "version" : "v1", + "timestamp" : "2013-09-02T00:00:00.000Z", + "event" : { + "count" : 1, + "language" : "en" + } +}, { + "version" : "v1", + "timestamp" : "2013-09-03T00:00:00.000Z", + "event" : { + "count" : 1, + "language" : "en" + } +} ] +``` + +if you set the origin for the granularity to `2012-01-01T00:30:00Z`, + +``` javascript + "granularity":{"type": "duration", "duration": "86400000", "origin":"2012-01-01T00:30:00Z"} +``` + +you will get + +``` json +[ { + "version" : "v1", + "timestamp" : "2013-08-31T00:30:00.000Z", + "event" : { + "count" : 1, + "language" : "en" + } +}, { + "version" : "v1", + "timestamp" : "2013-09-01T00:30:00.000Z", + "event" : { + "count" : 1, + "language" : "en" + } +}, { + "version" : "v1", + "timestamp" : "2013-09-02T00:30:00.000Z", + "event" : { + "count" : 1, + "language" : "en" + } +}, { + "version" : "v1", + "timestamp" : "2013-09-03T00:30:00.000Z", + "event" : { + "count" : 1, + "language" : "en" + } +} ] +``` + +Note that the timestamp for each bucket starts at the 30th minute. + ### Period Granularities Period granularities are specified as arbitrary period combinations of years, months, weeks, hours, minutes and seconds (e.g. P2W, P3M, PT1H30M, PT0.750S) in [ISO8601](https://en.wikipedia.org/wiki/ISO_8601) format. They support specifying a time zone which determines where period boundaries start as well as the timezone of the returned timestamps. By default, years start on the first of January, months start on the first of the month and weeks start on Mondays unless an origin is specified. @@ -52,5 +312,106 @@ This will bucket by two-day chunks in the Pacific timezone. This will bucket by 3-month chunks in the Pacific timezone where the three-month quarters are defined as starting from February. +#### Example + +Reusing the data in the previous example, if you submit a groupBy query with 1 day period in Pacific timezone, + +``` json +{ + "queryType":"groupBy", + "dataSource":"my_dataSource", + "granularity":{"type": "period", "period": "P1D", "timeZone": "America/Los_Angeles"}, + "dimensions":[ + "language" + ], + "aggregations":[ + { + "type":"count", + "name":"count" + } + ], + "intervals":[ + "1999-12-31T16:00:00.000-08:00/2999-12-31T16:00:00.000-08:00" + ] +} +``` + +you will get + +``` json +[ { + "version" : "v1", + "timestamp" : "2013-08-30T00:00:00.000-07:00", + "event" : { + "count" : 1, + "language" : "en" + } +}, { + "version" : "v1", + "timestamp" : "2013-08-31T00:00:00.000-07:00", + "event" : { + "count" : 1, + "language" : "en" + } +}, { + "version" : "v1", + "timestamp" : "2013-09-02T00:00:00.000-07:00", + "event" : { + "count" : 2, + "language" : "en" + } +} ] +``` + +Note that the timestamp for each bucket has been converted to Pacific time. Row `{"timestamp": "2013-09-02T23:32:45Z", "page": "CCC", "language" : "en"}` and +`{"timestamp": "2013-09-03T03:32:45Z", "page": "DDD", "language" : "en"}` are put in the same bucket because they are in the same day in Pacific time. + +Also note that the `intervals` in groupBy query will not be converted to the timezone specified, the timezone specified in granularity is only applied on the +query results. + +If you set the origin for the granularity to `1970-01-01T20:30:00-08:00`, + +``` javascript + "granularity":{"type": "period", "period": "P1D", "timeZone": "America/Los_Angeles", "origin": "1970-01-01T20:30:00-08:00"} +``` + +you will get + +``` json +[ { + "version" : "v1", + "timestamp" : "2013-08-29T20:30:00.000-07:00", + "event" : { + "count" : 1, + "language" : "en" + } +}, { + "version" : "v1", + "timestamp" : "2013-08-30T20:30:00.000-07:00", + "event" : { + "count" : 1, + "language" : "en" + } +}, { + "version" : "v1", + "timestamp" : "2013-09-01T20:30:00.000-07:00", + "event" : { + "count" : 1, + "language" : "en" + } +}, { + "version" : "v1", + "timestamp" : "2013-09-02T20:30:00.000-07:00", + "event" : { + "count" : 1, + "language" : "en" + } +} ] +``` + +Note that the `origin` you specified has nothing to do with the timezone, it only serves as a starting point for locating the very first granularity bucket. +In this case, Row `{"timestamp": "2013-09-02T23:32:45Z", "page": "CCC", "language" : "en"}` and `{"timestamp": "2013-09-03T03:32:45Z", "page": "DDD", "language" : "en"}` +are not in the same bucket. + #### Supported Time Zones Timezone support is provided by the [Joda Time library](http://www.joda.org), which uses the standard IANA time zones. See the [Joda Time supported timezones](http://joda-time.sourceforge.net/timezones.html). diff --git a/docs/content/querying/lookups.md b/docs/content/querying/lookups.md index 35f734dd9ddb..031b51eb2819 100644 --- a/docs/content/querying/lookups.md +++ b/docs/content/querying/lookups.md @@ -16,7 +16,7 @@ Very small lookups (count of keys on the order of a few dozen to a few hundred) Namespaced lookups are appropriate for lookups which are not possible to pass at query time due to their size, or are not desired to be passed at query time because the data is to reside in and be handled by the Druid servers. Namespaced lookups can be specified as part of the runtime properties file. The property is a list of the namespaces described as per the sections on this page. ```json - druid.query.extraction.namespaceList=\ + druid.query.extraction.namespace.lookups=\ [{ "type":"uri", "namespace":"some_uri_lookup","uri": "file:/tmp/prefix/",\ "namespaceParseSpec":\ {"format":"csv","columns":["key","value"]},\ @@ -27,7 +27,7 @@ Namespaced lookups are appropriate for lookups which are not possible to pass at ``` Proper funcitonality of Namespaced lookups requires the following extension to be loaded on the broker, peon, and historical nodes: -`io.druid.extensions:namespace-lookup` +`io.druid.extensions:druid-namespace-lookup` ## Cache Settings The following are settings used by the nodes which service queries when setting namespaces (broker, peon, historical) diff --git a/docs/content/toc.textile b/docs/content/toc.textile index 6e158fa401e3..3e695048006a 100644 --- a/docs/content/toc.textile +++ b/docs/content/toc.textile @@ -89,6 +89,7 @@ h2. Development ** "Geographic Queries":../development/geo.html ** "Select Query":../development/select-query.html ** "Approximate Histograms and Quantiles":../development/approximate-histograms.html +** "Datasketches based Aggregators":../development/datasketches-aggregators.html ** "Router node":../development/router.html ** "New Kafka Firehose":../development/kafka-simple-consumer-firehose.html diff --git a/extensions/datasketches/README.md b/extensions/datasketches/README.md new file mode 100644 index 000000000000..a934b622ec21 --- /dev/null +++ b/extensions/datasketches/README.md @@ -0,0 +1,9 @@ +This module provides druid aggregators based on http://datasketches.github.io/ . + +Credits: This module is a result of feedback and work done by following people. + +https://github.com/cheddar +https://github.com/himanshug +https://github.com/leerho +https://github.com/will-lauer + diff --git a/extensions/datasketches/pom.xml b/extensions/datasketches/pom.xml new file mode 100644 index 000000000000..85c280ffc17c --- /dev/null +++ b/extensions/datasketches/pom.xml @@ -0,0 +1,120 @@ + + + + + 4.0.0 + + io.druid.extensions + druid-datasketches + druid-datasketches + Druid Aggregators based on datasketches lib http://datasketches.github.io/ + + + io.druid + druid + 0.9.0-SNAPSHOT + ../../pom.xml + + + + + com.yahoo.datasketches + sketches-core + 0.2.2 + + + io.druid + druid-api + ${druid.api.version} + provided + + + io.druid + druid-processing + ${project.parent.version} + provided + + + + com.fasterxml.jackson.core + jackson-annotations + ${jackson.version} + provided + + + com.fasterxml.jackson.core + jackson-core + ${jackson.version} + provided + + + com.fasterxml.jackson.core + jackson-databind + ${jackson.version} + provided + + + com.fasterxml.jackson.datatype + jackson-datatype-guava + ${jackson.version} + provided + + + com.fasterxml.jackson.datatype + jackson-datatype-joda + ${jackson.version} + provided + + + com.fasterxml.jackson.dataformat + jackson-dataformat-smile + ${jackson.version} + provided + + + com.fasterxml.jackson.jaxrs + jackson-jaxrs-json-provider + ${jackson.version} + provided + + + com.fasterxml.jackson.jaxrs + jackson-jaxrs-smile-provider + ${jackson.version} + provided + + + + + junit + junit + test + + + io.druid + druid-processing + ${project.parent.version} + test-jar + test + + + + diff --git a/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/EmptySketchAggregator.java b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/EmptySketchAggregator.java new file mode 100644 index 000000000000..19ee610b53b1 --- /dev/null +++ b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/EmptySketchAggregator.java @@ -0,0 +1,71 @@ +/* +* Licensed to Metamarkets Group Inc. (Metamarkets) under one +* or more contributor license agreements. See the NOTICE file +* distributed with this work for additional information +* regarding copyright ownership. Metamarkets licenses this file +* to you under the Apache License, Version 2.0 (the +* "License"); you may not use this file except in compliance +* with the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, +* software distributed under the License is distributed on an +* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +* KIND, either express or implied. See the License for the +* specific language governing permissions and limitations +* under the License. +*/ + +package io.druid.query.aggregation.datasketches.theta; + +import io.druid.query.aggregation.Aggregator; + +public class EmptySketchAggregator implements Aggregator +{ + private final String name; + + public EmptySketchAggregator(String name) + { + this.name = name; + } + + @Override + public void aggregate() + { + } + + @Override + public void reset() + { + } + + @Override + public Object get() + { + return SketchOperations.EMPTY_SKETCH; + } + + @Override + public float getFloat() + { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public long getLong() + { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public String getName() + { + return name; + } + + @Override + public void close() + { + } +} diff --git a/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/EmptySketchBufferAggregator.java b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/EmptySketchBufferAggregator.java new file mode 100644 index 000000000000..160cc5f60b38 --- /dev/null +++ b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/EmptySketchBufferAggregator.java @@ -0,0 +1,64 @@ +/* +* Licensed to Metamarkets Group Inc. (Metamarkets) under one +* or more contributor license agreements. See the NOTICE file +* distributed with this work for additional information +* regarding copyright ownership. Metamarkets licenses this file +* to you under the Apache License, Version 2.0 (the +* "License"); you may not use this file except in compliance +* with the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, +* software distributed under the License is distributed on an +* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +* KIND, either express or implied. See the License for the +* specific language governing permissions and limitations +* under the License. +*/ + +package io.druid.query.aggregation.datasketches.theta; + +import io.druid.query.aggregation.BufferAggregator; + +import java.nio.ByteBuffer; + +public class EmptySketchBufferAggregator implements BufferAggregator +{ + public EmptySketchBufferAggregator() + { + } + + @Override + public void init(ByteBuffer buf, int position) + { + } + + @Override + public void aggregate(ByteBuffer buf, int position) + { + } + + @Override + public Object get(ByteBuffer buf, int position) + { + return SketchOperations.EMPTY_SKETCH; + } + + @Override + public float getFloat(ByteBuffer buf, int position) + { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public long getLong(ByteBuffer buf, int position) + { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public void close() + { + } +} diff --git a/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchAggregator.java b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchAggregator.java new file mode 100644 index 000000000000..e81b2ff978c7 --- /dev/null +++ b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchAggregator.java @@ -0,0 +1,130 @@ +/* +* Licensed to Metamarkets Group Inc. (Metamarkets) under one +* or more contributor license agreements. See the NOTICE file +* distributed with this work for additional information +* regarding copyright ownership. Metamarkets licenses this file +* to you under the Apache License, Version 2.0 (the +* "License"); you may not use this file except in compliance +* with the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, +* software distributed under the License is distributed on an +* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +* KIND, either express or implied. See the License for the +* specific language governing permissions and limitations +* under the License. +*/ + +package io.druid.query.aggregation.datasketches.theta; + +import com.metamx.common.ISE; +import com.metamx.common.logger.Logger; +import com.yahoo.sketches.Family; +import com.yahoo.sketches.memory.Memory; +import com.yahoo.sketches.theta.SetOperation; +import com.yahoo.sketches.theta.Sketch; +import com.yahoo.sketches.theta.Union; +import io.druid.query.aggregation.Aggregator; +import io.druid.segment.ObjectColumnSelector; + +import java.util.List; + +public class SketchAggregator implements Aggregator +{ + private static final Logger logger = new Logger(SketchAggregator.class); + + private final ObjectColumnSelector selector; + private final String name; + private final int size; + + private Union union; + + public SketchAggregator(String name, ObjectColumnSelector selector, int size) + { + this.name = name; + this.selector = selector; + this.size = size; + union = new SynchronizedUnion((Union) SetOperation.builder().build(size, Family.UNION)); + } + + @Override + public void aggregate() + { + Object update = selector.get(); + if (update == null) { + return; + } + + updateUnion(union, update); + } + + @Override + public void reset() + { + union.reset(); + } + + @Override + public Object get() + { + //in the code below, I am returning SetOp.getResult(true, null) + //"true" returns an ordered sketch but slower to compute than unordered sketch. + //however, advantage of ordered sketch is that they are faster to "union" later + //given that results from the aggregator will be combined further, it is better + //to return the ordered sketch here + return union.getResult(true, null); + } + + @Override + public float getFloat() + { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public long getLong() + { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public String getName() + { + return name; + } + + @Override + public void close() + { + union = null; + } + + static void updateUnion(Union union, Object update) + { + if (update instanceof Memory) { + union.update((Memory) update); + } else if (update instanceof Sketch) { + union.update((Sketch) update); + } else if (update instanceof String) { + union.update((String) update); + } else if (update instanceof byte[]) { + union.update((byte[]) update); + } else if (update instanceof Double) { + union.update(((Double) update)); + } else if (update instanceof Integer || update instanceof Long) { + union.update(((Number) update).longValue()); + } else if (update instanceof int[]) { + union.update((int[]) update); + } else if (update instanceof long[]) { + union.update((long[]) update); + } else if (update instanceof List) { + for (Object entry : (List) update) { + union.update(entry.toString()); + } + } else { + throw new ISE("Illegal type received while theta sketch merging [%s]", update.getClass()); + } + } +} diff --git a/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchAggregatorFactory.java b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchAggregatorFactory.java new file mode 100644 index 000000000000..7fd551026bcc --- /dev/null +++ b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchAggregatorFactory.java @@ -0,0 +1,220 @@ +/* +* Licensed to Metamarkets Group Inc. (Metamarkets) under one +* or more contributor license agreements. See the NOTICE file +* distributed with this work for additional information +* regarding copyright ownership. Metamarkets licenses this file +* to you under the Apache License, Version 2.0 (the +* "License"); you may not use this file except in compliance +* with the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, +* software distributed under the License is distributed on an +* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +* KIND, either express or implied. See the License for the +* specific language governing permissions and limitations +* under the License. +*/ + +package io.druid.query.aggregation.datasketches.theta; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import com.google.common.primitives.Doubles; +import com.metamx.common.IAE; +import com.yahoo.sketches.Family; +import com.yahoo.sketches.Util; +import com.yahoo.sketches.memory.Memory; +import com.yahoo.sketches.theta.SetOperation; +import com.yahoo.sketches.theta.Sketch; +import com.yahoo.sketches.theta.Sketches; +import com.yahoo.sketches.theta.Union; +import io.druid.query.aggregation.Aggregator; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.BufferAggregator; +import io.druid.segment.ColumnSelectorFactory; +import io.druid.segment.ObjectColumnSelector; + +import java.nio.ByteBuffer; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; + +public abstract class SketchAggregatorFactory implements AggregatorFactory +{ + public static final int DEFAULT_MAX_SKETCH_SIZE = 16384; + + protected final String name; + protected final String fieldName; + protected final int size; + private final byte cacheId; + + public static final Comparator COMPARATOR = new Comparator() + { + @Override + public int compare(Sketch o, Sketch o1) + { + return Doubles.compare(o.getEstimate(), o1.getEstimate()); + } + }; + + public SketchAggregatorFactory(String name, String fieldName, Integer size, byte cacheId) + { + this.name = Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name"); + this.fieldName = Preconditions.checkNotNull(fieldName, "Must have a valid, non-null fieldName"); + + this.size = size == null ? DEFAULT_MAX_SKETCH_SIZE : size; + Util.checkIfPowerOf2(this.size, "size"); + + this.cacheId = cacheId; + } + + @SuppressWarnings("unchecked") + @Override + public Aggregator factorize(ColumnSelectorFactory metricFactory) + { + ObjectColumnSelector selector = metricFactory.makeObjectColumnSelector(fieldName); + if (selector == null) { + return new EmptySketchAggregator(name); + } else { + return new SketchAggregator(name, selector, size); + } + } + + @SuppressWarnings("unchecked") + @Override + public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) + { + ObjectColumnSelector selector = metricFactory.makeObjectColumnSelector(fieldName); + if (selector == null) { + return new EmptySketchBufferAggregator(); + } else { + return new SketchBufferAggregator(selector, size, getMaxIntermediateSize()); + } + } + + @Override + public Object deserialize(Object object) + { + return SketchOperations.deserialize(object); + } + + @Override + public Comparator getComparator() + { + return COMPARATOR; + } + + @Override + public Object combine(Object lhs, Object rhs) + { + Union union = (Union) SetOperation.builder().build(size, Family.UNION); + updateUnion(union, lhs); + updateUnion(union, rhs); + return union.getResult(false, null); + } + + private void updateUnion(Union union, Object obj) + { + if (obj == null) { + return; + } else if (obj instanceof Memory) { + union.update((Memory) obj); + } else if (obj instanceof Sketch) { + union.update((Sketch) obj); + } else { + throw new IAE("Object of type [%s] can not be unioned", obj.getClass().getName()); + } + } + + @Override + @JsonProperty + public String getName() + { + return name; + } + + @JsonProperty + public String getFieldName() + { + return fieldName; + } + + @JsonProperty + public int getSize() + { + return size; + } + + @Override + public int getMaxIntermediateSize() + { + return SetOperation.getMaxUnionBytes(size); + } + + @Override + public Object getAggregatorStartValue() + { + return Sketches.updateSketchBuilder().build(size); + } + + @Override + public List requiredFields() + { + return Collections.singletonList(fieldName); + } + + @Override + public byte[] getCacheKey() + { + byte[] fieldNameBytes = fieldName.getBytes(); + return ByteBuffer.allocate(1 + fieldNameBytes.length).put(cacheId).put(fieldNameBytes).array(); + } + + @Override + public String toString() + { + return getClass().getSimpleName() + "{" + + "fieldName='" + fieldName + '\'' + + ", name='" + name + '\'' + + ", size=" + size + + '}'; + } + + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + SketchAggregatorFactory that = (SketchAggregatorFactory) o; + + if (size != that.size) { + return false; + } + if (cacheId != that.cacheId) { + return false; + } + if (!name.equals(that.name)) { + return false; + } + return fieldName.equals(that.fieldName); + + } + + @Override + public int hashCode() + { + int result = name.hashCode(); + result = 31 * result + fieldName.hashCode(); + result = 31 * result + size; + result = 31 * result + (int) cacheId; + return result; + } +} diff --git a/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchBufferAggregator.java b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchBufferAggregator.java new file mode 100644 index 000000000000..0ae9aef13cd6 --- /dev/null +++ b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchBufferAggregator.java @@ -0,0 +1,119 @@ +/* +* Licensed to Metamarkets Group Inc. (Metamarkets) under one +* or more contributor license agreements. See the NOTICE file +* distributed with this work for additional information +* regarding copyright ownership. Metamarkets licenses this file +* to you under the Apache License, Version 2.0 (the +* "License"); you may not use this file except in compliance +* with the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, +* software distributed under the License is distributed on an +* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +* KIND, either express or implied. See the License for the +* specific language governing permissions and limitations +* under the License. +*/ + +package io.druid.query.aggregation.datasketches.theta; + +import com.metamx.common.logger.Logger; +import com.yahoo.sketches.Family; +import com.yahoo.sketches.memory.Memory; +import com.yahoo.sketches.memory.MemoryRegion; +import com.yahoo.sketches.memory.NativeMemory; +import com.yahoo.sketches.theta.SetOperation; +import com.yahoo.sketches.theta.Union; +import io.druid.query.aggregation.BufferAggregator; +import io.druid.segment.ObjectColumnSelector; + +import java.nio.ByteBuffer; +import java.util.HashMap; +import java.util.Map; + +public class SketchBufferAggregator implements BufferAggregator +{ + private static final Logger logger = new Logger(SketchAggregator.class); + + private final ObjectColumnSelector selector; + private final int size; + private final int maxIntermediateSize; + + private NativeMemory nm; + + private final Map unions = new HashMap<>(); //position in BB -> Union Object + + public SketchBufferAggregator(ObjectColumnSelector selector, int size, int maxIntermediateSize) + { + this.selector = selector; + this.size = size; + this.maxIntermediateSize = maxIntermediateSize; + } + + @Override + public void init(ByteBuffer buf, int position) + { + if (nm == null) { + nm = new NativeMemory(buf); + } + + Memory mem = new MemoryRegion(nm, position, maxIntermediateSize); + unions.put(position, (Union) SetOperation.builder().initMemory(mem).build(size, Family.UNION)); + } + + @Override + public void aggregate(ByteBuffer buf, int position) + { + Object update = selector.get(); + if (update == null) { + return; + } + + Union union = getUnion(buf, position); + SketchAggregator.updateUnion(union, update); + } + + @Override + public Object get(ByteBuffer buf, int position) + { + //in the code below, I am returning SetOp.getResult(true, null) + //"true" returns an ordered sketch but slower to compute than unordered sketch. + //however, advantage of ordered sketch is that they are faster to "union" later + //given that results from the aggregator will be combined further, it is better + //to return the ordered sketch here + return getUnion(buf, position).getResult(true, null); + } + + //Note that this is not threadsafe and I don't think it needs to be + private Union getUnion(ByteBuffer buf, int position) + { + Union union = unions.get(position); + if (union == null) { + Memory mem = new MemoryRegion(nm, position, maxIntermediateSize); + union = (Union) SetOperation.wrap(mem); + unions.put(position, union); + } + return union; + } + + @Override + public float getFloat(ByteBuffer buf, int position) + { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public long getLong(ByteBuffer buf, int position) + { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public void close() + { + unions.clear(); + } + +} diff --git a/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchBuildComplexMetricSerde.java b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchBuildComplexMetricSerde.java new file mode 100644 index 000000000000..534d26024304 --- /dev/null +++ b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchBuildComplexMetricSerde.java @@ -0,0 +1,48 @@ +/* +* Licensed to Metamarkets Group Inc. (Metamarkets) under one +* or more contributor license agreements. See the NOTICE file +* distributed with this work for additional information +* regarding copyright ownership. Metamarkets licenses this file +* to you under the Apache License, Version 2.0 (the +* "License"); you may not use this file except in compliance +* with the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, +* software distributed under the License is distributed on an +* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +* KIND, either express or implied. See the License for the +* specific language governing permissions and limitations +* under the License. +*/ + +package io.druid.query.aggregation.datasketches.theta; + +import io.druid.data.input.InputRow; +import io.druid.segment.serde.ComplexMetricExtractor; + +/** + */ +public class SketchBuildComplexMetricSerde extends SketchMergeComplexMetricSerde +{ + @Override + public ComplexMetricExtractor getExtractor() + { + return new ComplexMetricExtractor() + { + + @Override + public Class extractedClass() + { + return Object.class; + } + + @Override + public Object extractValue(InputRow inputRow, String metricName) + { + return inputRow.getRaw(metricName); + } + }; + } +} diff --git a/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchEstimatePostAggregator.java b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchEstimatePostAggregator.java new file mode 100644 index 000000000000..b34203f2545b --- /dev/null +++ b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchEstimatePostAggregator.java @@ -0,0 +1,118 @@ +/* +* Licensed to Metamarkets Group Inc. (Metamarkets) under one +* or more contributor license agreements. See the NOTICE file +* distributed with this work for additional information +* regarding copyright ownership. Metamarkets licenses this file +* to you under the Apache License, Version 2.0 (the +* "License"); you may not use this file except in compliance +* with the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, +* software distributed under the License is distributed on an +* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +* KIND, either express or implied. See the License for the +* specific language governing permissions and limitations +* under the License. +*/ + +package io.druid.query.aggregation.datasketches.theta; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import com.google.common.collect.Sets; +import com.yahoo.sketches.theta.Sketch; +import io.druid.query.aggregation.PostAggregator; + +import java.util.Comparator; +import java.util.Map; +import java.util.Set; + +public class SketchEstimatePostAggregator implements PostAggregator +{ + + private final String name; + private final PostAggregator field; + + @JsonCreator + public SketchEstimatePostAggregator( + @JsonProperty("name") String name, + @JsonProperty("field") PostAggregator field + ) + { + this.name = Preconditions.checkNotNull(name, "name is null"); + this.field = Preconditions.checkNotNull(field, "field is null"); + } + + @Override + public Set getDependentFields() + { + Set dependentFields = Sets.newHashSet(); + dependentFields.addAll(field.getDependentFields()); + return dependentFields; + } + + @Override + public Comparator getComparator() + { + return SketchAggregatorFactory.COMPARATOR; + } + + @Override + public Object compute(Map combinedAggregators) + { + Sketch sketch = (Sketch) field.compute(combinedAggregators); + return sketch.getEstimate(); + } + + @Override + @JsonProperty + public String getName() + { + return name; + } + + @JsonProperty + public PostAggregator getField() + { + return field; + } + + @Override + public String toString() + { + return "SketchEstimatePostAggregator{" + + "name='" + name + '\'' + + ", field=" + field + + "}"; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + SketchEstimatePostAggregator that = (SketchEstimatePostAggregator) o; + + if (!name.equals(that.name)) { + return false; + } + return field.equals(that.field); + + } + + @Override + public int hashCode() + { + int result = name.hashCode(); + result = 31 * result + field.hashCode(); + return result; + } +} diff --git a/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchJsonSerializer.java b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchJsonSerializer.java new file mode 100644 index 000000000000..76db0d990560 --- /dev/null +++ b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchJsonSerializer.java @@ -0,0 +1,38 @@ +/* +* Licensed to Metamarkets Group Inc. (Metamarkets) under one +* or more contributor license agreements. See the NOTICE file +* distributed with this work for additional information +* regarding copyright ownership. Metamarkets licenses this file +* to you under the Apache License, Version 2.0 (the +* "License"); you may not use this file except in compliance +* with the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, +* software distributed under the License is distributed on an +* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +* KIND, either express or implied. See the License for the +* specific language governing permissions and limitations +* under the License. +*/ + +package io.druid.query.aggregation.datasketches.theta; + +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.JsonSerializer; +import com.fasterxml.jackson.databind.SerializerProvider; +import com.yahoo.sketches.theta.Sketch; + +import java.io.IOException; + +public class SketchJsonSerializer extends JsonSerializer +{ + @Override + public void serialize(Sketch sketch, JsonGenerator jgen, SerializerProvider provider) + throws IOException, JsonProcessingException + { + jgen.writeBinary(sketch.toByteArray()); + } +} diff --git a/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchMergeAggregatorFactory.java b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchMergeAggregatorFactory.java new file mode 100644 index 000000000000..4eec21b2cc7e --- /dev/null +++ b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchMergeAggregatorFactory.java @@ -0,0 +1,154 @@ +/* +* Licensed to Metamarkets Group Inc. (Metamarkets) under one +* or more contributor license agreements. See the NOTICE file +* distributed with this work for additional information +* regarding copyright ownership. Metamarkets licenses this file +* to you under the Apache License, Version 2.0 (the +* "License"); you may not use this file except in compliance +* with the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, +* software distributed under the License is distributed on an +* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +* KIND, either express or implied. See the License for the +* specific language governing permissions and limitations +* under the License. +*/ + +package io.druid.query.aggregation.datasketches.theta; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.yahoo.sketches.theta.Sketch; +import io.druid.query.aggregation.AggregatorFactory; + +import java.util.Collections; +import java.util.List; + +public class SketchMergeAggregatorFactory extends SketchAggregatorFactory +{ + + private static final byte CACHE_TYPE_ID = 15; + + private final boolean shouldFinalize; + private final boolean isInputThetaSketch; + + @JsonCreator + public SketchMergeAggregatorFactory( + @JsonProperty("name") String name, + @JsonProperty("fieldName") String fieldName, + @JsonProperty("size") Integer size, + @JsonProperty("shouldFinalize") Boolean shouldFinalize, + @JsonProperty("isInputThetaSketch") Boolean isInputThetaSketch + ) + { + super(name, fieldName, size, CACHE_TYPE_ID); + this.shouldFinalize = (shouldFinalize == null) ? true : shouldFinalize.booleanValue(); + this.isInputThetaSketch = (isInputThetaSketch == null) ? false : isInputThetaSketch.booleanValue(); + } + + @Override + public List getRequiredColumns() + { + return Collections.singletonList( + new SketchMergeAggregatorFactory( + fieldName, + fieldName, + size, + shouldFinalize, + isInputThetaSketch + ) + ); + } + + @Override + public AggregatorFactory getCombiningFactory() + { + return new SketchMergeAggregatorFactory(name, name, size, shouldFinalize, isInputThetaSketch); + } + + @JsonProperty + public boolean getShouldFinalize() + { + return shouldFinalize; + } + + @JsonProperty + public boolean getIsInputThetaSketch() + { + return isInputThetaSketch; + } + + /** + * Finalize the computation on sketch object and returns estimate from underlying + * sketch. + * + * @param object the sketch object + * + * @return sketch object + */ + @Override + public Object finalizeComputation(Object object) + { + if (shouldFinalize) { + return ((Sketch) object).getEstimate(); + } else { + return object; + } + } + + @Override + public String getTypeName() + { + if (isInputThetaSketch) { + return SketchModule.THETA_SKETCH_MERGE_AGG; + } else { + return SketchModule.THETA_SKETCH_BUILD_AGG; + } + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + + SketchMergeAggregatorFactory that = (SketchMergeAggregatorFactory) o; + + if (shouldFinalize != that.shouldFinalize) { + return false; + } + return isInputThetaSketch == that.isInputThetaSketch; + + } + + @Override + public int hashCode() + { + int result = super.hashCode(); + result = 31 * result + (shouldFinalize ? 1 : 0); + result = 31 * result + (isInputThetaSketch ? 1 : 0); + return result; + } + + @Override + public String toString() + { + return "SketchMergeAggregatorFactory{" + + "fieldName=" + fieldName + + ", name=" + name + + ", size=" + size + + ",shouldFinalize=" + shouldFinalize + + ", isInputThetaSketch=" + isInputThetaSketch + + "}"; + } +} diff --git a/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchMergeComplexMetricSerde.java b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchMergeComplexMetricSerde.java new file mode 100644 index 000000000000..e67c07f4abf7 --- /dev/null +++ b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchMergeComplexMetricSerde.java @@ -0,0 +1,83 @@ +/* +* Licensed to Metamarkets Group Inc. (Metamarkets) under one +* or more contributor license agreements. See the NOTICE file +* distributed with this work for additional information +* regarding copyright ownership. Metamarkets licenses this file +* to you under the Apache License, Version 2.0 (the +* "License"); you may not use this file except in compliance +* with the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, +* software distributed under the License is distributed on an +* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +* KIND, either express or implied. See the License for the +* specific language governing permissions and limitations +* under the License. +*/ + +package io.druid.query.aggregation.datasketches.theta; + +import com.yahoo.sketches.memory.Memory; +import com.yahoo.sketches.theta.Sketch; +import io.druid.data.input.InputRow; +import io.druid.segment.column.ColumnBuilder; +import io.druid.segment.data.GenericIndexed; +import io.druid.segment.data.ObjectStrategy; +import io.druid.segment.serde.ColumnPartSerde; +import io.druid.segment.serde.ComplexColumnPartSerde; +import io.druid.segment.serde.ComplexColumnPartSupplier; +import io.druid.segment.serde.ComplexMetricExtractor; +import io.druid.segment.serde.ComplexMetricSerde; + +import java.nio.ByteBuffer; + +public class SketchMergeComplexMetricSerde extends ComplexMetricSerde +{ + private SketchObjectStrategy strategy = new SketchObjectStrategy(); + + @Override + public String getTypeName() + { + return SketchModule.THETA_SKETCH; + } + + @Override + public ComplexMetricExtractor getExtractor() + { + return new ComplexMetricExtractor() + { + @Override + public Class extractedClass() + { + return Sketch.class; + } + + @Override + public Object extractValue(InputRow inputRow, String metricName) + { + final Object object = inputRow.getRaw(metricName); + if (object == null || object instanceof Sketch || object instanceof Memory) { + return object; + } + return SketchOperations.deserialize(object); + } + }; + } + + @Override + public ColumnPartSerde deserializeColumn(ByteBuffer buffer, ColumnBuilder builder) + { + GenericIndexed ge = GenericIndexed.read(buffer, strategy); + builder.setComplexColumn(new ComplexColumnPartSupplier(getTypeName(), ge)); + return new ComplexColumnPartSerde(ge, getTypeName()); + } + + @Override + public ObjectStrategy getObjectStrategy() + { + return strategy; + } + +} diff --git a/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchModule.java b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchModule.java new file mode 100644 index 000000000000..333b7f8a8b48 --- /dev/null +++ b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchModule.java @@ -0,0 +1,72 @@ +/* +* Licensed to Metamarkets Group Inc. (Metamarkets) under one +* or more contributor license agreements. See the NOTICE file +* distributed with this work for additional information +* regarding copyright ownership. Metamarkets licenses this file +* to you under the Apache License, Version 2.0 (the +* "License"); you may not use this file except in compliance +* with the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, +* software distributed under the License is distributed on an +* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +* KIND, either express or implied. See the License for the +* specific language governing permissions and limitations +* under the License. +*/ + +package io.druid.query.aggregation.datasketches.theta; + +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.jsontype.NamedType; +import com.fasterxml.jackson.databind.module.SimpleModule; +import com.google.inject.Binder; +import com.yahoo.sketches.theta.Sketch; +import io.druid.initialization.DruidModule; +import io.druid.segment.serde.ComplexMetrics; + +import java.util.Arrays; +import java.util.List; + +public class SketchModule implements DruidModule +{ + public static final String THETA_SKETCH = "thetaSketch"; + + public static final String THETA_SKETCH_MERGE_AGG = "thetaSketchMerge"; + public static final String THETA_SKETCH_BUILD_AGG = "thetaSketchBuild"; + + public static final String THETA_SKETCH_ESTIMATE_POST_AGG = "thetaSketchEstimate"; + public static final String THETA_SKETCH_SET_OP_POST_AGG = "thetaSketchSetOp"; + + @Override + public void configure(Binder binder) + { + if (ComplexMetrics.getSerdeForType(THETA_SKETCH) == null) { + ComplexMetrics.registerSerde(THETA_SKETCH, new SketchMergeComplexMetricSerde()); + } + + if (ComplexMetrics.getSerdeForType(THETA_SKETCH_MERGE_AGG) == null) { + ComplexMetrics.registerSerde(THETA_SKETCH_MERGE_AGG, new SketchMergeComplexMetricSerde()); + } + + if (ComplexMetrics.getSerdeForType(THETA_SKETCH_BUILD_AGG) == null) { + ComplexMetrics.registerSerde(THETA_SKETCH_BUILD_AGG, new SketchBuildComplexMetricSerde()); + } + } + + @Override + public List getJacksonModules() + { + return Arrays.asList( + new SimpleModule("ThetaSketchModule") + .registerSubtypes( + new NamedType(SketchMergeAggregatorFactory.class, THETA_SKETCH), + new NamedType(SketchEstimatePostAggregator.class, THETA_SKETCH_ESTIMATE_POST_AGG), + new NamedType(SketchSetPostAggregator.class, THETA_SKETCH_SET_OP_POST_AGG) + ) + .addSerializer(Sketch.class, new SketchJsonSerializer()) + ); + } +} diff --git a/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchObjectStrategy.java b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchObjectStrategy.java new file mode 100644 index 000000000000..c8ef75bb1222 --- /dev/null +++ b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchObjectStrategy.java @@ -0,0 +1,107 @@ +/* +* Licensed to Metamarkets Group Inc. (Metamarkets) under one +* or more contributor license agreements. See the NOTICE file +* distributed with this work for additional information +* regarding copyright ownership. Metamarkets licenses this file +* to you under the Apache License, Version 2.0 (the +* "License"); you may not use this file except in compliance +* with the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, +* software distributed under the License is distributed on an +* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +* KIND, either express or implied. See the License for the +* specific language governing permissions and limitations +* under the License. +*/ + +package io.druid.query.aggregation.datasketches.theta; + +import com.google.common.primitives.Longs; +import com.metamx.common.IAE; +import com.yahoo.sketches.memory.Memory; +import com.yahoo.sketches.memory.MemoryRegion; +import com.yahoo.sketches.memory.NativeMemory; +import com.yahoo.sketches.theta.Sketch; +import com.yahoo.sketches.theta.Sketches; +import io.druid.segment.data.ObjectStrategy; + +import java.nio.ByteBuffer; + +public class SketchObjectStrategy implements ObjectStrategy +{ + + private static final byte[] EMPTY_BYTES = new byte[]{}; + private static final Sketch EMPTY_SKETCH = Sketches.updateSketchBuilder().build().compact(true, null); + + @Override + public int compare(Object s1, Object s2) + { + if (s1 instanceof Sketch) { + if (s2 instanceof Sketch) { + return SketchAggregatorFactory.COMPARATOR.compare((Sketch) s1, (Sketch) s2); + } else { + return -1; + } + } + if (s1 instanceof Memory) { + if (s2 instanceof Memory) { + Memory s1Mem = (Memory) s1; + Memory s2Mem = (Memory) s2; + + // We have two Ordered Compact sketches, so just compare their last entry if they have the size. + // This is to produce a deterministic ordering, though it might not match the actual estimate + // ordering, but that's ok because this comparator is only used by GenericIndexed + int retVal = Longs.compare(s1Mem.getCapacity(), s2Mem.getCapacity()); + if (retVal == 0) { + retVal = Longs.compare(s1Mem.getLong(s1Mem.getCapacity() - 8), s2Mem.getLong(s2Mem.getCapacity() - 8)); + } + + return retVal; + } else { + return 1; + } + } + throw new IAE("Unknwon class[%s], toString[%s]", s1.getClass(), s1); + + } + + @Override + public Class getClazz() + { + return Sketch.class; + } + + @Override + public Object fromByteBuffer(ByteBuffer buffer, int numBytes) + { + if (numBytes == 0) { + return EMPTY_SKETCH; + } + + return new MemoryRegion(new NativeMemory(buffer), buffer.position(), numBytes); + } + + @Override + public byte[] toBytes(Object obj) + { + if (obj instanceof Sketch) { + Sketch sketch = (Sketch) obj; + if (sketch.isEmpty()) { + return EMPTY_BYTES; + } + return sketch.toByteArray(); + } else if (obj instanceof Memory) { + Memory mem = (Memory) obj; + byte[] retVal = new byte[(int) mem.getCapacity()]; + mem.getByteArray(0, retVal, 0, (int) mem.getCapacity()); + return retVal; + } else if (obj == null) { + return EMPTY_BYTES; + } else { + throw new IAE("Unknown class[%s], toString[%s]", obj.getClass(), obj); + } + } +} diff --git a/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchOperations.java b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchOperations.java new file mode 100644 index 000000000000..e6e6e2ca142b --- /dev/null +++ b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchOperations.java @@ -0,0 +1,123 @@ +/* +* Licensed to Metamarkets Group Inc. (Metamarkets) under one +* or more contributor license agreements. See the NOTICE file +* distributed with this work for additional information +* regarding copyright ownership. Metamarkets licenses this file +* to you under the Apache License, Version 2.0 (the +* "License"); you may not use this file except in compliance +* with the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, +* software distributed under the License is distributed on an +* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +* KIND, either express or implied. See the License for the +* specific language governing permissions and limitations +* under the License. +*/ + +package io.druid.query.aggregation.datasketches.theta; + +import com.google.common.base.Charsets; +import com.metamx.common.logger.Logger; +import com.yahoo.sketches.Family; +import com.yahoo.sketches.memory.NativeMemory; +import com.yahoo.sketches.theta.AnotB; +import com.yahoo.sketches.theta.Intersection; +import com.yahoo.sketches.theta.SetOperation; +import com.yahoo.sketches.theta.Sketch; +import com.yahoo.sketches.theta.Sketches; +import com.yahoo.sketches.theta.Union; +import org.apache.commons.codec.binary.Base64; + +public class SketchOperations +{ + + private static final Logger LOG = new Logger(SketchOperations.class); + + public static final Sketch EMPTY_SKETCH = Sketches.updateSketchBuilder().build().compact(true, null); + + public static enum Func + { + UNION, + INTERSECT, + NOT; + } + + public static Sketch deserialize(Object serializedSketch) + { + if (serializedSketch instanceof String) { + return deserializeFromBase64EncodedString((String) serializedSketch); + } else if (serializedSketch instanceof byte[]) { + return deserializeFromByteArray((byte[]) serializedSketch); + } else if (serializedSketch instanceof Sketch) { + return (Sketch) serializedSketch; + } + + throw new IllegalStateException( + "Object is not of a type that can deserialize to sketch: " + + serializedSketch.getClass() + ); + } + + public static Sketch deserializeFromBase64EncodedString(String str) + { + return deserializeFromByteArray( + Base64.decodeBase64( + str.getBytes(Charsets.UTF_8) + ) + ); + } + + public static Sketch deserializeFromByteArray(byte[] data) + { + NativeMemory mem = new NativeMemory(data); + if (Sketch.getSerializationVersion(mem) < 3) { + return Sketches.heapifySketch(mem); + } else { + return Sketches.wrapSketch(mem); + } + } + + public static Sketch sketchSetOperation(Func func, int sketchSize, Sketch... sketches) + { + //in the code below, I am returning SetOp.getResult(false, null) + //"false" gets us an unordered sketch which is faster to build + //"true" returns an ordered sketch but slower to compute. advantage of ordered sketch + //is that they are faster to "union" later but given that this method is used in + //the final stages of query processing, ordered sketch would be of no use. + switch (func) { + case UNION: + Union union = (Union) SetOperation.builder().build(sketchSize, Family.UNION); + for (Sketch sketch : sketches) { + union.update(sketch); + } + return union.getResult(false, null); + case INTERSECT: + Intersection intersection = (Intersection) SetOperation.builder().build(sketchSize, Family.INTERSECTION); + for (Sketch sketch : sketches) { + intersection.update(sketch); + } + return intersection.getResult(false, null); + case NOT: + if (sketches.length < 1) { + throw new IllegalArgumentException("A-Not-B requires atleast 1 sketch"); + } + + if (sketches.length == 1) { + return sketches[0]; + } + + Sketch result = sketches[0]; + for (int i = 1; i < sketches.length; i++) { + AnotB anotb = (AnotB) SetOperation.builder().build(sketchSize, Family.A_NOT_B); + anotb.update(result, sketches[i]); + result = anotb.getResult(false, null); + } + return result; + default: + throw new IllegalArgumentException("Unknown sketch operation " + func); + } + } +} diff --git a/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchSetPostAggregator.java b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchSetPostAggregator.java new file mode 100644 index 000000000000..550fe2f55573 --- /dev/null +++ b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchSetPostAggregator.java @@ -0,0 +1,167 @@ +/* +* Licensed to Metamarkets Group Inc. (Metamarkets) under one +* or more contributor license agreements. See the NOTICE file +* distributed with this work for additional information +* regarding copyright ownership. Metamarkets licenses this file +* to you under the Apache License, Version 2.0 (the +* "License"); you may not use this file except in compliance +* with the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, +* software distributed under the License is distributed on an +* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +* KIND, either express or implied. See the License for the +* specific language governing permissions and limitations +* under the License. +*/ + +package io.druid.query.aggregation.datasketches.theta; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.Sets; +import com.metamx.common.IAE; +import com.metamx.common.logger.Logger; +import com.yahoo.sketches.Util; +import com.yahoo.sketches.theta.Sketch; +import io.druid.query.aggregation.PostAggregator; + +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.Set; + +public class SketchSetPostAggregator implements PostAggregator +{ + + private static final Logger LOG = new Logger(SketchSetPostAggregator.class); + + private final String name; + private final List fields; + private final SketchOperations.Func func; + private final int maxSketchSize; + + @JsonCreator + public SketchSetPostAggregator( + @JsonProperty("name") String name, + @JsonProperty("func") String func, + @JsonProperty("size") Integer maxSize, + @JsonProperty("fields") List fields + ) + { + this.name = name; + this.fields = fields; + this.func = SketchOperations.Func.valueOf(func); + this.maxSketchSize = maxSize == null ? SketchAggregatorFactory.DEFAULT_MAX_SKETCH_SIZE : maxSize; + Util.checkIfPowerOf2(this.maxSketchSize, "size"); + + if (fields.size() <= 1) { + throw new IAE("Illegal number of fields[%s], must be > 1", fields.size()); + } + } + + @Override + public Set getDependentFields() + { + Set dependentFields = Sets.newLinkedHashSet(); + for (PostAggregator field : fields) { + dependentFields.addAll(field.getDependentFields()); + } + return dependentFields; + } + + @Override + public Comparator getComparator() + { + return SketchAggregatorFactory.COMPARATOR; + } + + @Override + public Object compute(final Map combinedAggregators) + { + Sketch[] sketches = new Sketch[fields.size()]; + for (int i = 0; i < sketches.length; i++) { + sketches[i] = (Sketch) fields.get(i).compute(combinedAggregators); + } + + return SketchOperations.sketchSetOperation(func, maxSketchSize, sketches); + } + + @Override + @JsonProperty + public String getName() + { + return name; + } + + @JsonProperty + public String getFunc() + { + return func.toString(); + } + + @JsonProperty + public List getFields() + { + return fields; + } + + @JsonProperty + public int getSize() + { + return maxSketchSize; + } + + @Override + public String toString() + { + return "SketchSetPostAggregator{" + + "name='" + + name + + '\'' + + ", fields=" + + fields + + ", func=" + + func + + ", size=" + + maxSketchSize + + "}"; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + SketchSetPostAggregator that = (SketchSetPostAggregator) o; + + if (maxSketchSize != that.maxSketchSize) { + return false; + } + if (!name.equals(that.name)) { + return false; + } + if (!fields.equals(that.fields)) { + return false; + } + return func == that.func; + + } + + @Override + public int hashCode() + { + int result = name.hashCode(); + result = 31 * result + fields.hashCode(); + result = 31 * result + func.hashCode(); + result = 31 * result + maxSketchSize; + return result; + } +} diff --git a/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SynchronizedUnion.java b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SynchronizedUnion.java new file mode 100644 index 000000000000..f0af78a23e1f --- /dev/null +++ b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SynchronizedUnion.java @@ -0,0 +1,109 @@ +/* +* Licensed to Metamarkets Group Inc. (Metamarkets) under one +* or more contributor license agreements. See the NOTICE file +* distributed with this work for additional information +* regarding copyright ownership. Metamarkets licenses this file +* to you under the Apache License, Version 2.0 (the +* "License"); you may not use this file except in compliance +* with the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, +* software distributed under the License is distributed on an +* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +* KIND, either express or implied. See the License for the +* specific language governing permissions and limitations +* under the License. +*/ + +package io.druid.query.aggregation.datasketches.theta; + +import com.yahoo.sketches.memory.Memory; +import com.yahoo.sketches.theta.CompactSketch; +import com.yahoo.sketches.theta.Sketch; +import com.yahoo.sketches.theta.Union; + +/** + */ +public class SynchronizedUnion implements Union +{ + private final Union delegate; + + public SynchronizedUnion(Union delegate) + { + this.delegate = delegate; + } + + @Override + public synchronized void update(Sketch sketchIn) + { + delegate.update(sketchIn); + } + + @Override + public synchronized void update(Memory mem) + { + delegate.update(mem); + } + + @Override + public synchronized void update(long datum) + { + delegate.update(datum); + } + + @Override + public synchronized void update(double datum) + { + delegate.update(datum); + } + + @Override + public synchronized void update(String datum) + { + delegate.update(datum); + } + + @Override + public synchronized void update(byte[] data) + { + delegate.update(data); + } + + @Override + public synchronized void update(int[] data) + { + delegate.update(data); + } + + @Override + public synchronized void update(long[] data) + { + delegate.update(data); + } + + @Override + public synchronized CompactSketch getResult(boolean b, Memory memory) + { + return delegate.getResult(b, memory); + } + + @Override + public synchronized CompactSketch getResult() + { + return delegate.getResult(); + } + + @Override + public synchronized byte[] toByteArray() + { + return delegate.toByteArray(); + } + + @Override + public synchronized void reset() + { + delegate.reset(); + } +} diff --git a/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/oldapi/OldApiSketchModule.java b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/oldapi/OldApiSketchModule.java new file mode 100644 index 000000000000..f9182d2bc7d5 --- /dev/null +++ b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/oldapi/OldApiSketchModule.java @@ -0,0 +1,86 @@ +/* +* Licensed to Metamarkets Group Inc. (Metamarkets) under one +* or more contributor license agreements. See the NOTICE file +* distributed with this work for additional information +* regarding copyright ownership. Metamarkets licenses this file +* to you under the Apache License, Version 2.0 (the +* "License"); you may not use this file except in compliance +* with the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, +* software distributed under the License is distributed on an +* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +* KIND, either express or implied. See the License for the +* specific language governing permissions and limitations +* under the License. +*/ + +package io.druid.query.aggregation.datasketches.theta.oldapi; + +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.jsontype.NamedType; +import com.fasterxml.jackson.databind.module.SimpleModule; +import com.google.inject.Binder; +import com.yahoo.sketches.theta.Sketch; +import io.druid.initialization.DruidModule; +import io.druid.query.aggregation.datasketches.theta.SketchBuildComplexMetricSerde; +import io.druid.query.aggregation.datasketches.theta.SketchJsonSerializer; +import io.druid.query.aggregation.datasketches.theta.SketchMergeComplexMetricSerde; +import io.druid.query.aggregation.datasketches.theta.SketchModule; +import io.druid.segment.serde.ComplexMetrics; + +import java.util.Arrays; +import java.util.List; + +public class OldApiSketchModule implements DruidModule +{ + + public static final String SET_SKETCH = "setSketch"; + public static final String SKETCH_BUILD = "sketchBuild"; + public static final String SKETCH_MERGE = "sketchMerge"; + + @Override + public void configure(Binder binder) + { + if (ComplexMetrics.getSerdeForType(SKETCH_BUILD) == null) { + ComplexMetrics.registerSerde(SKETCH_BUILD, new SketchBuildComplexMetricSerde()); + } + + if (ComplexMetrics.getSerdeForType(SET_SKETCH) == null) { + ComplexMetrics.registerSerde(SET_SKETCH, new SketchMergeComplexMetricSerde()); + } + + if (ComplexMetrics.getSerdeForType(SKETCH_MERGE) == null) { + ComplexMetrics.registerSerde(SKETCH_MERGE, new SketchMergeComplexMetricSerde()); + } + + if (ComplexMetrics.getSerdeForType(SketchModule.THETA_SKETCH) == null) { + ComplexMetrics.registerSerde(SketchModule.THETA_SKETCH, new SketchMergeComplexMetricSerde()); + } + + if (ComplexMetrics.getSerdeForType(SketchModule.THETA_SKETCH_MERGE_AGG) == null) { + ComplexMetrics.registerSerde(SketchModule.THETA_SKETCH_MERGE_AGG, new SketchMergeComplexMetricSerde()); + } + + if (ComplexMetrics.getSerdeForType(SketchModule.THETA_SKETCH_BUILD_AGG) == null) { + ComplexMetrics.registerSerde(SketchModule.THETA_SKETCH_BUILD_AGG, new SketchBuildComplexMetricSerde()); + } + } + + @Override + public List getJacksonModules() + { + return Arrays.asList( + new SimpleModule("OldThetaSketchModule") + .registerSubtypes( + new NamedType(OldSketchBuildAggregatorFactory.class, SKETCH_BUILD), + new NamedType(OldSketchMergeAggregatorFactory.class, SKETCH_MERGE), + new NamedType(OldSketchEstimatePostAggregator.class, "sketchEstimate"), + new NamedType(OldSketchSetPostAggregator.class, "sketchSetOper") + ) + .addSerializer(Sketch.class, new SketchJsonSerializer()) + ); + } +} diff --git a/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/oldapi/OldSketchBuildAggregatorFactory.java b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/oldapi/OldSketchBuildAggregatorFactory.java new file mode 100644 index 000000000000..5b5bc3346994 --- /dev/null +++ b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/oldapi/OldSketchBuildAggregatorFactory.java @@ -0,0 +1,39 @@ +/* +* Licensed to Metamarkets Group Inc. (Metamarkets) under one +* or more contributor license agreements. See the NOTICE file +* distributed with this work for additional information +* regarding copyright ownership. Metamarkets licenses this file +* to you under the Apache License, Version 2.0 (the +* "License"); you may not use this file except in compliance +* with the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, +* software distributed under the License is distributed on an +* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +* KIND, either express or implied. See the License for the +* specific language governing permissions and limitations +* under the License. +*/ + +package io.druid.query.aggregation.datasketches.theta.oldapi; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.druid.query.aggregation.datasketches.theta.SketchMergeAggregatorFactory; + +/** + */ +public class OldSketchBuildAggregatorFactory extends SketchMergeAggregatorFactory +{ + @JsonCreator + public OldSketchBuildAggregatorFactory( + @JsonProperty("name") String name, + @JsonProperty("fieldName") String fieldName, + @JsonProperty("size") Integer size + ) + { + super(name, fieldName, size, true, false); + } +} diff --git a/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/oldapi/OldSketchEstimatePostAggregator.java b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/oldapi/OldSketchEstimatePostAggregator.java new file mode 100644 index 000000000000..1f0fa58ffbb7 --- /dev/null +++ b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/oldapi/OldSketchEstimatePostAggregator.java @@ -0,0 +1,39 @@ +/* +* Licensed to Metamarkets Group Inc. (Metamarkets) under one +* or more contributor license agreements. See the NOTICE file +* distributed with this work for additional information +* regarding copyright ownership. Metamarkets licenses this file +* to you under the Apache License, Version 2.0 (the +* "License"); you may not use this file except in compliance +* with the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, +* software distributed under the License is distributed on an +* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +* KIND, either express or implied. See the License for the +* specific language governing permissions and limitations +* under the License. +*/ + +package io.druid.query.aggregation.datasketches.theta.oldapi; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.druid.query.aggregation.PostAggregator; +import io.druid.query.aggregation.datasketches.theta.SketchEstimatePostAggregator; + +/** + */ +public class OldSketchEstimatePostAggregator extends SketchEstimatePostAggregator +{ + @JsonCreator + public OldSketchEstimatePostAggregator( + @JsonProperty("name") String name, + @JsonProperty("field") PostAggregator field + ) + { + super(name, field); + } +} diff --git a/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/oldapi/OldSketchMergeAggregatorFactory.java b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/oldapi/OldSketchMergeAggregatorFactory.java new file mode 100644 index 000000000000..82beb7a35373 --- /dev/null +++ b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/oldapi/OldSketchMergeAggregatorFactory.java @@ -0,0 +1,40 @@ +/* +* Licensed to Metamarkets Group Inc. (Metamarkets) under one +* or more contributor license agreements. See the NOTICE file +* distributed with this work for additional information +* regarding copyright ownership. Metamarkets licenses this file +* to you under the Apache License, Version 2.0 (the +* "License"); you may not use this file except in compliance +* with the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, +* software distributed under the License is distributed on an +* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +* KIND, either express or implied. See the License for the +* specific language governing permissions and limitations +* under the License. +*/ + +package io.druid.query.aggregation.datasketches.theta.oldapi; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.druid.query.aggregation.datasketches.theta.SketchMergeAggregatorFactory; + +/** + */ +public class OldSketchMergeAggregatorFactory extends SketchMergeAggregatorFactory +{ + @JsonCreator + public OldSketchMergeAggregatorFactory( + @JsonProperty("name") String name, + @JsonProperty("fieldName") String fieldName, + @JsonProperty("size") Integer size, + @JsonProperty("shouldFinalize") Boolean shouldFinalize + ) + { + super(name, fieldName, size, shouldFinalize, true); + } +} diff --git a/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/oldapi/OldSketchSetPostAggregator.java b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/oldapi/OldSketchSetPostAggregator.java new file mode 100644 index 000000000000..dee6a5cccbc5 --- /dev/null +++ b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/oldapi/OldSketchSetPostAggregator.java @@ -0,0 +1,43 @@ +/* +* Licensed to Metamarkets Group Inc. (Metamarkets) under one +* or more contributor license agreements. See the NOTICE file +* distributed with this work for additional information +* regarding copyright ownership. Metamarkets licenses this file +* to you under the Apache License, Version 2.0 (the +* "License"); you may not use this file except in compliance +* with the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, +* software distributed under the License is distributed on an +* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +* KIND, either express or implied. See the License for the +* specific language governing permissions and limitations +* under the License. +*/ + +package io.druid.query.aggregation.datasketches.theta.oldapi; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.druid.query.aggregation.PostAggregator; +import io.druid.query.aggregation.datasketches.theta.SketchSetPostAggregator; + +import java.util.List; + +/** + */ +public class OldSketchSetPostAggregator extends SketchSetPostAggregator +{ + @JsonCreator + public OldSketchSetPostAggregator( + @JsonProperty("name") String name, + @JsonProperty("func") String func, + @JsonProperty("size") Integer maxSize, + @JsonProperty("fields") List fields + ) + { + super(name, func, maxSize, fields); + } +} diff --git a/extensions/datasketches/src/main/resources/META-INF/services/io.druid.initialization.DruidModule b/extensions/datasketches/src/main/resources/META-INF/services/io.druid.initialization.DruidModule new file mode 100644 index 000000000000..efd10f3a7e78 --- /dev/null +++ b/extensions/datasketches/src/main/resources/META-INF/services/io.druid.initialization.DruidModule @@ -0,0 +1,2 @@ +io.druid.query.aggregation.datasketches.theta.SketchModule +io.druid.query.aggregation.datasketches.theta.oldapi.OldApiSketchModule diff --git a/extensions/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/SketchAggregationTest.java b/extensions/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/SketchAggregationTest.java new file mode 100644 index 000000000000..38671f363edf --- /dev/null +++ b/extensions/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/SketchAggregationTest.java @@ -0,0 +1,242 @@ +/* +* Licensed to Metamarkets Group Inc. (Metamarkets) under one +* or more contributor license agreements. See the NOTICE file +* distributed with this work for additional information +* regarding copyright ownership. Metamarkets licenses this file +* to you under the Apache License, Version 2.0 (the +* "License"); you may not use this file except in compliance +* with the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, +* software distributed under the License is distributed on an +* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +* KIND, either express or implied. See the License for the +* specific language governing permissions and limitations +* under the License. +*/ + +package io.druid.query.aggregation.datasketches.theta; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import com.google.common.io.Files; +import com.metamx.common.guava.Sequence; +import com.metamx.common.guava.Sequences; +import com.yahoo.sketches.theta.Sketch; +import com.yahoo.sketches.theta.Sketches; +import io.druid.data.input.MapBasedRow; +import io.druid.granularity.QueryGranularity; +import io.druid.query.aggregation.AggregationTestHelper; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.PostAggregator; +import io.druid.query.aggregation.post.FieldAccessPostAggregator; +import org.joda.time.DateTime; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.io.IOException; +import java.nio.charset.Charset; +import java.util.List; + +/** + */ +public class SketchAggregationTest +{ + private final AggregationTestHelper helper; + + @Rule + public final TemporaryFolder tempFolder = new TemporaryFolder(); + + public SketchAggregationTest() + { + SketchModule sm = new SketchModule(); + sm.configure(null); + helper = new AggregationTestHelper(sm.getJacksonModules(), tempFolder); + } + + @Test + public void testSimpleDataIngestAndQuery() throws Exception + { + Sequence seq = helper.createIndexAndRunQueryOnSegment( + new File(this.getClass().getClassLoader().getResource("simple_test_data.tsv").getFile()), + readFileFromClasspathAsString("simple_test_data_record_parser.json"), + readFileFromClasspathAsString("simple_test_data_aggregators.json"), + 0, + QueryGranularity.NONE, + 5, + readFileFromClasspathAsString("simple_test_data_group_by_query.json") + ); + + List results = Sequences.toList(seq, Lists.newArrayList()); + Assert.assertEquals(1, results.size()); + Assert.assertEquals( + new MapBasedRow( + DateTime.parse("2014-10-19T00:00:00.000Z"), + ImmutableMap + .builder() + .put("sketch_count", 50.0) + .put("sketchEstimatePostAgg", 50.0) + .put("sketchUnionPostAggEstimate", 50.0) + .put("sketchIntersectionPostAggEstimate", 50.0) + .put("sketchAnotBPostAggEstimate", 0.0) + .put("non_existing_col_validation", 0.0) + .build() + ), + results.get(0) + ); + } + + @Test + public void testSketchDataIngestAndQuery() throws Exception + { + Sequence seq = helper.createIndexAndRunQueryOnSegment( + new File(SketchAggregationTest.class.getClassLoader().getResource("sketch_test_data.tsv").getFile()), + readFileFromClasspathAsString("sketch_test_data_record_parser.json"), + readFileFromClasspathAsString("sketch_test_data_aggregators.json"), + 0, + QueryGranularity.NONE, + 5, + readFileFromClasspathAsString("sketch_test_data_group_by_query.json") + ); + + List results = Sequences.toList(seq, Lists.newArrayList()); + Assert.assertEquals(1, results.size()); + Assert.assertEquals( + new MapBasedRow( + DateTime.parse("2014-10-19T00:00:00.000Z"), + ImmutableMap + .builder() + .put("sids_sketch_count", 50.0) + .put("sketchEstimatePostAgg", 50.0) + .put("sketchUnionPostAggEstimate", 50.0) + .put("sketchIntersectionPostAggEstimate", 50.0) + .put("sketchAnotBPostAggEstimate", 0.0) + .put("non_existing_col_validation", 0.0) + .build() + ), + results.get(0) + ); + } + + @Test + public void testThetaCardinalityOnSimpleColumn() throws Exception + { + Sequence seq = helper.createIndexAndRunQueryOnSegment( + new File(SketchAggregationTest.class.getClassLoader().getResource("simple_test_data.tsv").getFile()), + readFileFromClasspathAsString("simple_test_data_record_parser2.json"), + "[" + + " {" + + " \"type\": \"count\"," + + " \"name\": \"count\"" + + " }" + + "]", + 0, + QueryGranularity.NONE, + 5, + readFileFromClasspathAsString("simple_test_data_group_by_query.json") + ); + + List results = Sequences.toList(seq, Lists.newArrayList()); + Assert.assertEquals(1, results.size()); + Assert.assertEquals( + new MapBasedRow( + DateTime.parse("2014-10-19T00:00:00.000Z"), + ImmutableMap + .builder() + .put("sketch_count", 50.0) + .put("sketchEstimatePostAgg", 50.0) + .put("sketchUnionPostAggEstimate", 50.0) + .put("sketchIntersectionPostAggEstimate", 50.0) + .put("sketchAnotBPostAggEstimate", 0.0) + .put("non_existing_col_validation", 0.0) + .build() + ), + results.get(0) + ); + } + + @Test + public void testSketchMergeAggregatorFactorySerde() throws Exception + { + assertAggregatorFactorySerde(new SketchMergeAggregatorFactory("name", "fieldName", 16, null, null)); + assertAggregatorFactorySerde(new SketchMergeAggregatorFactory("name", "fieldName", 16, false, true)); + assertAggregatorFactorySerde(new SketchMergeAggregatorFactory("name", "fieldName", 16, true, false)); + } + + @Test + public void testSketchMergeFinalization() throws Exception + { + Sketch sketch = Sketches.updateSketchBuilder().build(128); + + SketchMergeAggregatorFactory agg = new SketchMergeAggregatorFactory("name", "fieldName", 16, null, null); + Assert.assertEquals(0.0, ((Double) agg.finalizeComputation(sketch)).doubleValue(), 0.0001); + + agg = new SketchMergeAggregatorFactory("name", "fieldName", 16, true, null); + Assert.assertEquals(0.0, ((Double) agg.finalizeComputation(sketch)).doubleValue(), 0.0001); + + agg = new SketchMergeAggregatorFactory("name", "fieldName", 16, false, null); + Assert.assertEquals(sketch, agg.finalizeComputation(sketch)); + } + + private void assertAggregatorFactorySerde(AggregatorFactory agg) throws Exception + { + Assert.assertEquals( + agg, + helper.getObjectMapper().readValue( + helper.getObjectMapper().writeValueAsString(agg), + AggregatorFactory.class + ) + ); + } + + @Test + public void testSketchEstimatePostAggregatorSerde() throws Exception + { + assertPostAggregatorSerde( + new SketchEstimatePostAggregator( + "name", + new FieldAccessPostAggregator("name", "fieldName") + ) + ); + } + + @Test + public void testSketchSetPostAggregatorSerde() throws Exception + { + assertPostAggregatorSerde( + new SketchSetPostAggregator( + "name", + "INTERSECT", + null, + Lists.newArrayList( + new FieldAccessPostAggregator("name1", "fieldName1"), + new FieldAccessPostAggregator("name2", "fieldName2") + ) + ) + ); + } + + private void assertPostAggregatorSerde(PostAggregator agg) throws Exception + { + Assert.assertEquals( + agg, + helper.getObjectMapper().readValue( + helper.getObjectMapper().writeValueAsString(agg), + PostAggregator.class + ) + ); + } + + public final static String readFileFromClasspathAsString(String fileName) throws IOException + { + return Files.asCharSource( + new File(SketchAggregationTest.class.getClassLoader().getResource(fileName).getFile()), + Charset.forName("UTF-8") + ).read(); + } +} diff --git a/extensions/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/oldapi/OldApiSketchAggregationTest.java b/extensions/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/oldapi/OldApiSketchAggregationTest.java new file mode 100644 index 000000000000..c683f6c10dd0 --- /dev/null +++ b/extensions/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/oldapi/OldApiSketchAggregationTest.java @@ -0,0 +1,198 @@ +/* +* Licensed to Metamarkets Group Inc. (Metamarkets) under one +* or more contributor license agreements. See the NOTICE file +* distributed with this work for additional information +* regarding copyright ownership. Metamarkets licenses this file +* to you under the Apache License, Version 2.0 (the +* "License"); you may not use this file except in compliance +* with the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, +* software distributed under the License is distributed on an +* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +* KIND, either express or implied. See the License for the +* specific language governing permissions and limitations +* under the License. +*/ + +package io.druid.query.aggregation.datasketches.theta.oldapi; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import com.google.common.io.Files; +import com.metamx.common.guava.Sequence; +import com.metamx.common.guava.Sequences; +import io.druid.data.input.MapBasedRow; +import io.druid.granularity.QueryGranularity; +import io.druid.query.aggregation.AggregationTestHelper; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.PostAggregator; +import io.druid.query.aggregation.post.FieldAccessPostAggregator; +import org.joda.time.DateTime; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.io.IOException; +import java.nio.charset.Charset; +import java.util.List; + +/** + */ +public class OldApiSketchAggregationTest +{ + private final AggregationTestHelper helper; + + @Rule + public final TemporaryFolder tempFolder = new TemporaryFolder(); + + public OldApiSketchAggregationTest() + { + OldApiSketchModule sm = new OldApiSketchModule(); + sm.configure(null); + + helper = new AggregationTestHelper( + sm.getJacksonModules(), + tempFolder + ); + } + + @Test + public void testSimpleDataIngestAndQuery() throws Exception + { + Sequence seq = helper.createIndexAndRunQueryOnSegment( + new File(this.getClass().getClassLoader().getResource("simple_test_data.tsv").getFile()), + readFileFromClasspathAsString("simple_test_data_record_parser.json"), + readFileFromClasspathAsString("oldapi/old_simple_test_data_aggregators.json"), + 0, + QueryGranularity.NONE, + 5, + readFileFromClasspathAsString("oldapi/old_simple_test_data_group_by_query.json") + ); + + List results = Sequences.toList(seq, Lists.newArrayList()); + Assert.assertEquals(1, results.size()); + Assert.assertEquals( + new MapBasedRow( + DateTime.parse("2014-10-19T00:00:00.000Z"), + ImmutableMap + .builder() + .put("sketch_count", 50.0) + .put("sketchEstimatePostAgg", 50.0) + .put("sketchUnionPostAggEstimate", 50.0) + .put("sketchIntersectionPostAggEstimate", 50.0) + .put("sketchAnotBPostAggEstimate", 0.0) + .put("non_existing_col_validation", 0.0) + .build() + ), + results.get(0) + ); + } + + @Test + public void testSketchDataIngestAndQuery() throws Exception + { + Sequence seq = helper.createIndexAndRunQueryOnSegment( + new File(OldApiSketchAggregationTest.class.getClassLoader().getResource("sketch_test_data.tsv").getFile()), + readFileFromClasspathAsString("sketch_test_data_record_parser.json"), + readFileFromClasspathAsString("oldapi/old_sketch_test_data_aggregators.json"), + 0, + QueryGranularity.NONE, + 5, + readFileFromClasspathAsString("oldapi/old_sketch_test_data_group_by_query.json") + ); + + List results = Sequences.toList(seq, Lists.newArrayList()); + Assert.assertEquals(1, results.size()); + Assert.assertEquals( + new MapBasedRow( + DateTime.parse("2014-10-19T00:00:00.000Z"), + ImmutableMap + .builder() + .put("sids_sketch_count", 50.0) + .put("sketchEstimatePostAgg", 50.0) + .put("sketchUnionPostAggEstimate", 50.0) + .put("sketchIntersectionPostAggEstimate", 50.0) + .put("sketchAnotBPostAggEstimate", 0.0) + .put("non_existing_col_validation", 0.0) + .build() + ), + results.get(0) + ); + } + + @Test + public void testSketchMergeAggregatorFactorySerde() throws Exception + { + assertAggregatorFactorySerde(new OldSketchMergeAggregatorFactory("name", "fieldName", 16, null)); + assertAggregatorFactorySerde(new OldSketchMergeAggregatorFactory("name", "fieldName", 16, false)); + assertAggregatorFactorySerde(new OldSketchMergeAggregatorFactory("name", "fieldName", 16, true)); + } + + @Test + public void testSketchBuildAggregatorFactorySerde() throws Exception + { + assertAggregatorFactorySerde(new OldSketchBuildAggregatorFactory("name", "fieldName", 16)); + } + + private void assertAggregatorFactorySerde(AggregatorFactory agg) throws Exception + { + Assert.assertEquals( + agg, + helper.getObjectMapper().readValue( + helper.getObjectMapper().writeValueAsString(agg), + AggregatorFactory.class + ) + ); + } + + @Test + public void testSketchEstimatePostAggregatorSerde() throws Exception + { + assertPostAggregatorSerde( + new OldSketchEstimatePostAggregator( + "name", + new FieldAccessPostAggregator("name", "fieldName") + ) + ); + } + + @Test + public void testSketchSetPostAggregatorSerde() throws Exception + { + assertPostAggregatorSerde( + new OldSketchSetPostAggregator( + "name", + "INTERSECT", + null, + Lists.newArrayList( + new FieldAccessPostAggregator("name1", "fieldName1"), + new FieldAccessPostAggregator("name2", "fieldName2") + ) + ) + ); + } + + private void assertPostAggregatorSerde(PostAggregator agg) throws Exception + { + Assert.assertEquals( + agg, + helper.getObjectMapper().readValue( + helper.getObjectMapper().writeValueAsString(agg), + PostAggregator.class + ) + ); + } + + public final static String readFileFromClasspathAsString(String fileName) throws IOException + { + return Files.asCharSource( + new File(OldApiSketchAggregationTest.class.getClassLoader().getResource(fileName).getFile()), + Charset.forName("UTF-8") + ).read(); + } +} diff --git a/extensions/datasketches/src/test/resources/oldapi/old_simple_test_data_aggregators.json b/extensions/datasketches/src/test/resources/oldapi/old_simple_test_data_aggregators.json new file mode 100644 index 000000000000..fbefe20e2d24 --- /dev/null +++ b/extensions/datasketches/src/test/resources/oldapi/old_simple_test_data_aggregators.json @@ -0,0 +1,12 @@ +[ + { + "type": "sketchBuild", + "name": "pty_country", + "fieldName": "pty_country" + }, + { + "type": "sketchBuild", + "name": "non_existing_col_validation", + "fieldName": "non_existing_col" + } +] diff --git a/extensions/datasketches/src/test/resources/oldapi/old_simple_test_data_group_by_query.json b/extensions/datasketches/src/test/resources/oldapi/old_simple_test_data_group_by_query.json new file mode 100644 index 000000000000..362652cac339 --- /dev/null +++ b/extensions/datasketches/src/test/resources/oldapi/old_simple_test_data_group_by_query.json @@ -0,0 +1,93 @@ +{ + "queryType": "groupBy", + "dataSource": "test_datasource", + "granularity": "ALL", + "dimensions": [], + "aggregations": [ + { + "type": "sketchMerge", + "name": "sketch_count", + "fieldName": "pty_country", + "size": 16384 + }, + { + "type": "sketchMerge", + "name": "non_existing_col_validation", + "fieldName": "non_existing_col", + "size": 16384 + } + ], + "postAggregations": [ + { + "type": "sketchEstimate", + "name": "sketchEstimatePostAgg", + "field": { + "type": "fieldAccess", + "fieldName": "sketch_count" + } + }, + { + "type": "sketchEstimate", + "name": "sketchIntersectionPostAggEstimate", + "field": { + "type": "sketchSetOper", + "name": "sketchIntersectionPostAgg", + "func": "INTERSECT", + "size": 16384, + "fields": [ + { + "type": "fieldAccess", + "fieldName": "sketch_count" + }, + { + "type": "fieldAccess", + "fieldName": "sketch_count" + } + ] + } + }, + { + "type": "sketchEstimate", + "name": "sketchAnotBPostAggEstimate", + "field": { + "type": "sketchSetOper", + "name": "sketchAnotBUnionPostAgg", + "func": "NOT", + "size": 16384, + "fields": [ + { + "type": "fieldAccess", + "fieldName": "sketch_count" + }, + { + "type": "fieldAccess", + "fieldName": "sketch_count" + } + ] + } + }, + { + "type": "sketchEstimate", + "name": "sketchUnionPostAggEstimate", + "field": { + "type": "sketchSetOper", + "name": "sketchUnionPostAgg", + "func": "UNION", + "size": 16384, + "fields": [ + { + "type": "fieldAccess", + "fieldName": "sketch_count" + }, + { + "type": "fieldAccess", + "fieldName": "sketch_count" + } + ] + } + } + ], + "intervals": [ + "2014-10-19T00:00:00.000Z/2014-10-22T00:00:00.000Z" + ] +} diff --git a/extensions/datasketches/src/test/resources/oldapi/old_sketch_test_data_aggregators.json b/extensions/datasketches/src/test/resources/oldapi/old_sketch_test_data_aggregators.json new file mode 100644 index 000000000000..7cd10e17678d --- /dev/null +++ b/extensions/datasketches/src/test/resources/oldapi/old_sketch_test_data_aggregators.json @@ -0,0 +1,14 @@ +[ + { + "type": "sketchMerge", + "name": "sids_sketch", + "fieldName": "sketch", + "size": 16384 + }, + { + "type": "sketchMerge", + "name": "non_existing_col_validation", + "fieldName": "non_existing_col", + "size": 16384 + } +] diff --git a/extensions/datasketches/src/test/resources/oldapi/old_sketch_test_data_group_by_query.json b/extensions/datasketches/src/test/resources/oldapi/old_sketch_test_data_group_by_query.json new file mode 100644 index 000000000000..027e0a610cb6 --- /dev/null +++ b/extensions/datasketches/src/test/resources/oldapi/old_sketch_test_data_group_by_query.json @@ -0,0 +1,92 @@ +{ + "queryType": "groupBy", + "dataSource": "test_datasource", + "granularity": "ALL", + "dimensions": [], + "aggregations": [ + { + "type": "sketchMerge", + "name": "sids_sketch_count", + "fieldName": "sids_sketch", + "size": 16384 + }, + { + "type": "sketchMerge", + "name": "non_existing_col_validation", + "fieldName": "non_existing_col", + "size": 16384 + } + ], + "postAggregations": [ + { + "type": "sketchEstimate", + "name": "sketchEstimatePostAgg", + "field": { + "type": "fieldAccess", + "fieldName": "sids_sketch_count" + } + }, + { + "type": "sketchEstimate", + "name": "sketchIntersectionPostAggEstimate", + "field": { + "type": "sketchSetOper", + "name": "sketchIntersectionPostAgg", + "func": "INTERSECT", + "size": 16384, + "fields": [ + { + "type": "fieldAccess", + "fieldName": "sids_sketch_count" + }, + { + "type": "fieldAccess", + "fieldName": "sids_sketch_count" + } + ] + } + }, + { + "type": "sketchEstimate", + "name": "sketchAnotBPostAggEstimate", + "field": { + "type": "sketchSetOper", + "name": "sketchAnotBUnionPostAgg", + "func": "NOT", + "fields": [ + { + "type": "fieldAccess", + "fieldName": "sids_sketch_count" + }, + { + "type": "fieldAccess", + "fieldName": "sids_sketch_count" + } + ] + } + }, + { + "type": "sketchEstimate", + "name": "sketchUnionPostAggEstimate", + "field": { + "type": "sketchSetOper", + "name": "sketchUnionPostAgg", + "func": "UNION", + "size": 16384, + "fields": [ + { + "type": "fieldAccess", + "fieldName": "sids_sketch_count" + }, + { + "type": "fieldAccess", + "fieldName": "sids_sketch_count" + } + ] + } + } + ], + "intervals": [ + "2014-10-19T00:00:00.000Z/2014-10-22T00:00:00.000Z" + ] +} diff --git a/extensions/datasketches/src/test/resources/simple_test_data.tsv b/extensions/datasketches/src/test/resources/simple_test_data.tsv new file mode 100644 index 000000000000..361f350905d7 --- /dev/null +++ b/extensions/datasketches/src/test/resources/simple_test_data.tsv @@ -0,0 +1,3360 @@ +2014102000 product_1 pty_country_34 +2014102000 product_14 pty_country_8 +2014102000 product_10 pty_country_39 +2014102000 product_7 pty_country_6 +2014102000 product_15 pty_country_28 +2014102000 product_14 pty_country_44 +2014102000 product_11 pty_country_16 +2014102000 product_13 pty_country_12 +2014102000 product_5 pty_country_31 +2014102000 product_12 pty_country_9 +2014102001 product_10 pty_country_29 +2014102001 product_1 pty_country_23 +2014102001 product_14 pty_country_5 +2014102001 product_10 pty_country_17 +2014102001 product_3 pty_country_8 +2014102001 product_1 pty_country_13 +2014102001 product_8 pty_country_12 +2014102001 product_1 pty_country_34 +2014102001 product_9 pty_country_6 +2014102001 product_3 pty_country_34 +2014102002 product_7 pty_country_44 +2014102002 product_7 pty_country_16 +2014102002 product_5 pty_country_1 +2014102002 product_10 pty_country_8 +2014102002 product_14 pty_country_43 +2014102002 product_6 pty_country_17 +2014102002 product_8 pty_country_43 +2014102002 product_15 pty_country_8 +2014102002 product_4 pty_country_37 +2014102002 product_8 pty_country_4 +2014102003 product_10 pty_country_42 +2014102003 product_9 pty_country_34 +2014102003 product_2 pty_country_14 +2014102003 product_4 pty_country_18 +2014102003 product_15 pty_country_39 +2014102003 product_14 pty_country_8 +2014102003 product_13 pty_country_17 +2014102003 product_6 pty_country_12 +2014102003 product_9 pty_country_49 +2014102003 product_8 pty_country_5 +2014102004 product_8 pty_country_35 +2014102004 product_14 pty_country_4 +2014102004 product_7 pty_country_5 +2014102004 product_5 pty_country_16 +2014102004 product_5 pty_country_4 +2014102004 product_4 pty_country_43 +2014102004 product_2 pty_country_20 +2014102004 product_5 pty_country_3 +2014102004 product_8 pty_country_48 +2014102004 product_7 pty_country_29 +2014102005 product_7 pty_country_19 +2014102005 product_15 pty_country_6 +2014102005 product_3 pty_country_1 +2014102005 product_4 pty_country_39 +2014102005 product_10 pty_country_42 +2014102005 product_4 pty_country_14 +2014102005 product_13 pty_country_41 +2014102005 product_9 pty_country_34 +2014102005 product_4 pty_country_42 +2014102005 product_7 pty_country_34 +2014102006 product_7 pty_country_6 +2014102006 product_1 pty_country_36 +2014102006 product_4 pty_country_27 +2014102006 product_3 pty_country_4 +2014102006 product_5 pty_country_14 +2014102006 product_2 pty_country_48 +2014102006 product_11 pty_country_21 +2014102006 product_13 pty_country_26 +2014102006 product_8 pty_country_1 +2014102006 product_4 pty_country_31 +2014102007 product_4 pty_country_5 +2014102007 product_11 pty_country_18 +2014102007 product_13 pty_country_48 +2014102007 product_2 pty_country_1 +2014102007 product_6 pty_country_12 +2014102007 product_11 pty_country_37 +2014102007 product_14 pty_country_43 +2014102007 product_8 pty_country_35 +2014102007 product_7 pty_country_18 +2014102007 product_12 pty_country_19 +2014102008 product_14 pty_country_30 +2014102008 product_3 pty_country_22 +2014102008 product_6 pty_country_46 +2014102008 product_10 pty_country_25 +2014102008 product_15 pty_country_32 +2014102008 product_11 pty_country_8 +2014102008 product_4 pty_country_7 +2014102008 product_1 pty_country_28 +2014102008 product_7 pty_country_26 +2014102008 product_9 pty_country_21 +2014102009 product_10 pty_country_23 +2014102009 product_12 pty_country_3 +2014102009 product_6 pty_country_17 +2014102009 product_2 pty_country_27 +2014102009 product_3 pty_country_11 +2014102009 product_1 pty_country_30 +2014102009 product_12 pty_country_25 +2014102009 product_9 pty_country_43 +2014102009 product_3 pty_country_44 +2014102009 product_7 pty_country_17 +2014102010 product_6 pty_country_16 +2014102010 product_4 pty_country_41 +2014102010 product_12 pty_country_44 +2014102010 product_1 pty_country_9 +2014102010 product_2 pty_country_12 +2014102010 product_9 pty_country_20 +2014102010 product_4 pty_country_16 +2014102010 product_5 pty_country_28 +2014102010 product_12 pty_country_46 +2014102010 product_9 pty_country_15 +2014102011 product_10 pty_country_12 +2014102011 product_6 pty_country_35 +2014102011 product_1 pty_country_43 +2014102011 product_1 pty_country_15 +2014102011 product_4 pty_country_2 +2014102011 product_9 pty_country_11 +2014102011 product_6 pty_country_50 +2014102011 product_10 pty_country_18 +2014102011 product_15 pty_country_22 +2014102011 product_8 pty_country_20 +2014102012 product_7 pty_country_14 +2014102012 product_8 pty_country_37 +2014102012 product_6 pty_country_37 +2014102012 product_13 pty_country_17 +2014102012 product_9 pty_country_31 +2014102012 product_3 pty_country_22 +2014102012 product_8 pty_country_29 +2014102012 product_3 pty_country_8 +2014102012 product_10 pty_country_17 +2014102012 product_7 pty_country_20 +2014102013 product_7 pty_country_33 +2014102013 product_8 pty_country_46 +2014102013 product_5 pty_country_44 +2014102013 product_11 pty_country_25 +2014102013 product_1 pty_country_8 +2014102013 product_2 pty_country_23 +2014102013 product_7 pty_country_45 +2014102013 product_14 pty_country_8 +2014102013 product_8 pty_country_5 +2014102013 product_12 pty_country_45 +2014102014 product_13 pty_country_36 +2014102014 product_10 pty_country_14 +2014102014 product_15 pty_country_38 +2014102014 product_9 pty_country_20 +2014102014 product_3 pty_country_28 +2014102014 product_5 pty_country_25 +2014102014 product_12 pty_country_30 +2014102014 product_9 pty_country_49 +2014102014 product_8 pty_country_10 +2014102014 product_4 pty_country_50 +2014102015 product_8 pty_country_29 +2014102015 product_3 pty_country_10 +2014102015 product_6 pty_country_34 +2014102015 product_11 pty_country_47 +2014102015 product_14 pty_country_16 +2014102015 product_1 pty_country_12 +2014102015 product_5 pty_country_19 +2014102015 product_15 pty_country_31 +2014102015 product_11 pty_country_25 +2014102015 product_11 pty_country_3 +2014102016 product_8 pty_country_6 +2014102016 product_7 pty_country_32 +2014102016 product_8 pty_country_12 +2014102016 product_9 pty_country_41 +2014102016 product_8 pty_country_29 +2014102016 product_6 pty_country_19 +2014102016 product_9 pty_country_30 +2014102016 product_9 pty_country_41 +2014102016 product_4 pty_country_47 +2014102016 product_1 pty_country_24 +2014102017 product_2 pty_country_47 +2014102017 product_3 pty_country_32 +2014102017 product_8 pty_country_44 +2014102017 product_2 pty_country_38 +2014102017 product_14 pty_country_2 +2014102017 product_5 pty_country_9 +2014102017 product_8 pty_country_19 +2014102017 product_5 pty_country_10 +2014102017 product_8 pty_country_14 +2014102017 product_12 pty_country_34 +2014102018 product_12 pty_country_26 +2014102018 product_5 pty_country_40 +2014102018 product_5 pty_country_31 +2014102018 product_10 pty_country_34 +2014102018 product_14 pty_country_8 +2014102018 product_11 pty_country_5 +2014102018 product_1 pty_country_9 +2014102018 product_2 pty_country_26 +2014102018 product_11 pty_country_31 +2014102018 product_1 pty_country_17 +2014102019 product_13 pty_country_43 +2014102019 product_5 pty_country_25 +2014102019 product_5 pty_country_16 +2014102019 product_8 pty_country_32 +2014102019 product_4 pty_country_7 +2014102019 product_12 pty_country_2 +2014102019 product_1 pty_country_35 +2014102019 product_15 pty_country_40 +2014102019 product_1 pty_country_39 +2014102019 product_4 pty_country_14 +2014102020 product_7 pty_country_37 +2014102020 product_11 pty_country_13 +2014102020 product_7 pty_country_40 +2014102020 product_7 pty_country_1 +2014102020 product_5 pty_country_9 +2014102020 product_14 pty_country_39 +2014102020 product_7 pty_country_34 +2014102020 product_5 pty_country_28 +2014102020 product_4 pty_country_3 +2014102020 product_10 pty_country_27 +2014102021 product_8 pty_country_1 +2014102021 product_13 pty_country_25 +2014102021 product_3 pty_country_26 +2014102021 product_6 pty_country_25 +2014102021 product_15 pty_country_13 +2014102021 product_15 pty_country_50 +2014102021 product_10 pty_country_27 +2014102021 product_6 pty_country_34 +2014102021 product_13 pty_country_41 +2014102021 product_9 pty_country_42 +2014102022 product_7 pty_country_37 +2014102022 product_13 pty_country_50 +2014102022 product_12 pty_country_22 +2014102022 product_13 pty_country_17 +2014102022 product_13 pty_country_33 +2014102022 product_15 pty_country_38 +2014102022 product_13 pty_country_4 +2014102022 product_9 pty_country_39 +2014102022 product_7 pty_country_33 +2014102022 product_15 pty_country_18 +2014102023 product_6 pty_country_43 +2014102023 product_6 pty_country_20 +2014102023 product_15 pty_country_13 +2014102023 product_2 pty_country_20 +2014102023 product_13 pty_country_32 +2014102023 product_11 pty_country_19 +2014102023 product_4 pty_country_22 +2014102023 product_15 pty_country_38 +2014102023 product_4 pty_country_20 +2014102023 product_3 pty_country_15 +2014102100 product_5 pty_country_5 +2014102100 product_14 pty_country_44 +2014102100 product_9 pty_country_45 +2014102100 product_4 pty_country_5 +2014102100 product_7 pty_country_2 +2014102100 product_10 pty_country_49 +2014102100 product_9 pty_country_24 +2014102100 product_3 pty_country_44 +2014102100 product_1 pty_country_2 +2014102100 product_13 pty_country_31 +2014102101 product_4 pty_country_50 +2014102101 product_8 pty_country_5 +2014102101 product_8 pty_country_34 +2014102101 product_4 pty_country_24 +2014102101 product_14 pty_country_34 +2014102101 product_9 pty_country_17 +2014102101 product_7 pty_country_8 +2014102101 product_9 pty_country_39 +2014102101 product_13 pty_country_9 +2014102101 product_4 pty_country_28 +2014102102 product_10 pty_country_16 +2014102102 product_2 pty_country_37 +2014102102 product_12 pty_country_1 +2014102102 product_3 pty_country_49 +2014102102 product_3 pty_country_47 +2014102102 product_6 pty_country_29 +2014102102 product_9 pty_country_24 +2014102102 product_1 pty_country_27 +2014102102 product_12 pty_country_47 +2014102102 product_2 pty_country_43 +2014102103 product_15 pty_country_46 +2014102103 product_15 pty_country_6 +2014102103 product_13 pty_country_8 +2014102103 product_14 pty_country_1 +2014102103 product_5 pty_country_17 +2014102103 product_9 pty_country_31 +2014102103 product_5 pty_country_34 +2014102103 product_9 pty_country_20 +2014102103 product_6 pty_country_36 +2014102103 product_13 pty_country_5 +2014102104 product_2 pty_country_15 +2014102104 product_12 pty_country_20 +2014102104 product_15 pty_country_43 +2014102104 product_12 pty_country_23 +2014102104 product_6 pty_country_30 +2014102104 product_6 pty_country_50 +2014102104 product_1 pty_country_3 +2014102104 product_15 pty_country_4 +2014102104 product_3 pty_country_1 +2014102104 product_4 pty_country_45 +2014102105 product_11 pty_country_42 +2014102105 product_2 pty_country_49 +2014102105 product_13 pty_country_43 +2014102105 product_1 pty_country_12 +2014102105 product_9 pty_country_37 +2014102105 product_8 pty_country_6 +2014102105 product_2 pty_country_6 +2014102105 product_8 pty_country_36 +2014102105 product_3 pty_country_17 +2014102105 product_10 pty_country_44 +2014102106 product_15 pty_country_1 +2014102106 product_6 pty_country_11 +2014102106 product_9 pty_country_32 +2014102106 product_10 pty_country_43 +2014102106 product_6 pty_country_45 +2014102106 product_12 pty_country_34 +2014102106 product_13 pty_country_1 +2014102106 product_5 pty_country_37 +2014102106 product_4 pty_country_44 +2014102106 product_14 pty_country_18 +2014102107 product_2 pty_country_29 +2014102107 product_2 pty_country_20 +2014102107 product_7 pty_country_22 +2014102107 product_9 pty_country_46 +2014102107 product_14 pty_country_2 +2014102107 product_5 pty_country_16 +2014102107 product_15 pty_country_4 +2014102107 product_12 pty_country_45 +2014102107 product_14 pty_country_11 +2014102107 product_9 pty_country_27 +2014102108 product_2 pty_country_18 +2014102108 product_8 pty_country_23 +2014102108 product_10 pty_country_5 +2014102108 product_2 pty_country_5 +2014102108 product_6 pty_country_43 +2014102108 product_5 pty_country_19 +2014102108 product_7 pty_country_8 +2014102108 product_6 pty_country_3 +2014102108 product_8 pty_country_34 +2014102108 product_13 pty_country_23 +2014102109 product_14 pty_country_8 +2014102109 product_4 pty_country_43 +2014102109 product_11 pty_country_10 +2014102109 product_2 pty_country_33 +2014102109 product_1 pty_country_41 +2014102109 product_11 pty_country_39 +2014102109 product_13 pty_country_31 +2014102109 product_15 pty_country_23 +2014102109 product_5 pty_country_38 +2014102109 product_2 pty_country_32 +2014102110 product_12 pty_country_44 +2014102110 product_1 pty_country_2 +2014102110 product_13 pty_country_14 +2014102110 product_14 pty_country_3 +2014102110 product_10 pty_country_5 +2014102110 product_10 pty_country_28 +2014102110 product_14 pty_country_44 +2014102110 product_13 pty_country_26 +2014102110 product_5 pty_country_40 +2014102110 product_10 pty_country_26 +2014102111 product_10 pty_country_48 +2014102111 product_13 pty_country_23 +2014102111 product_3 pty_country_28 +2014102111 product_9 pty_country_40 +2014102111 product_5 pty_country_43 +2014102111 product_14 pty_country_12 +2014102111 product_7 pty_country_16 +2014102111 product_14 pty_country_11 +2014102111 product_2 pty_country_35 +2014102111 product_12 pty_country_43 +2014102112 product_7 pty_country_22 +2014102112 product_4 pty_country_10 +2014102112 product_6 pty_country_41 +2014102112 product_11 pty_country_28 +2014102112 product_6 pty_country_2 +2014102112 product_7 pty_country_49 +2014102112 product_10 pty_country_8 +2014102112 product_5 pty_country_18 +2014102112 product_12 pty_country_16 +2014102112 product_14 pty_country_29 +2014102113 product_7 pty_country_35 +2014102113 product_9 pty_country_45 +2014102113 product_8 pty_country_2 +2014102113 product_14 pty_country_49 +2014102113 product_6 pty_country_50 +2014102113 product_7 pty_country_49 +2014102113 product_12 pty_country_17 +2014102113 product_6 pty_country_17 +2014102113 product_4 pty_country_8 +2014102113 product_8 pty_country_44 +2014102114 product_6 pty_country_33 +2014102114 product_15 pty_country_7 +2014102114 product_9 pty_country_40 +2014102114 product_12 pty_country_13 +2014102114 product_12 pty_country_21 +2014102114 product_14 pty_country_12 +2014102114 product_9 pty_country_7 +2014102114 product_5 pty_country_47 +2014102114 product_8 pty_country_25 +2014102114 product_14 pty_country_34 +2014102115 product_11 pty_country_8 +2014102115 product_8 pty_country_29 +2014102115 product_1 pty_country_22 +2014102115 product_1 pty_country_44 +2014102115 product_2 pty_country_26 +2014102115 product_14 pty_country_1 +2014102115 product_5 pty_country_32 +2014102115 product_14 pty_country_39 +2014102115 product_7 pty_country_38 +2014102115 product_5 pty_country_28 +2014102116 product_7 pty_country_50 +2014102116 product_7 pty_country_47 +2014102116 product_3 pty_country_34 +2014102116 product_5 pty_country_47 +2014102116 product_12 pty_country_8 +2014102116 product_3 pty_country_4 +2014102116 product_15 pty_country_28 +2014102116 product_9 pty_country_29 +2014102116 product_8 pty_country_20 +2014102116 product_6 pty_country_12 +2014102117 product_13 pty_country_25 +2014102117 product_4 pty_country_44 +2014102117 product_11 pty_country_13 +2014102117 product_2 pty_country_43 +2014102117 product_8 pty_country_13 +2014102117 product_7 pty_country_16 +2014102117 product_10 pty_country_14 +2014102117 product_1 pty_country_38 +2014102117 product_11 pty_country_21 +2014102117 product_11 pty_country_1 +2014102118 product_7 pty_country_49 +2014102118 product_12 pty_country_21 +2014102118 product_11 pty_country_21 +2014102118 product_5 pty_country_41 +2014102118 product_9 pty_country_31 +2014102118 product_5 pty_country_49 +2014102118 product_3 pty_country_30 +2014102118 product_7 pty_country_4 +2014102118 product_11 pty_country_33 +2014102118 product_14 pty_country_5 +2014102119 product_4 pty_country_48 +2014102119 product_8 pty_country_22 +2014102119 product_13 pty_country_35 +2014102119 product_12 pty_country_8 +2014102119 product_13 pty_country_12 +2014102119 product_6 pty_country_34 +2014102119 product_4 pty_country_27 +2014102119 product_11 pty_country_24 +2014102119 product_3 pty_country_14 +2014102119 product_12 pty_country_38 +2014102120 product_5 pty_country_10 +2014102120 product_14 pty_country_37 +2014102120 product_4 pty_country_50 +2014102120 product_13 pty_country_11 +2014102120 product_1 pty_country_40 +2014102120 product_10 pty_country_43 +2014102120 product_5 pty_country_26 +2014102120 product_3 pty_country_26 +2014102120 product_5 pty_country_33 +2014102120 product_12 pty_country_32 +2014102121 product_12 pty_country_16 +2014102121 product_9 pty_country_35 +2014102121 product_3 pty_country_12 +2014102121 product_4 pty_country_3 +2014102121 product_4 pty_country_40 +2014102121 product_2 pty_country_30 +2014102121 product_10 pty_country_11 +2014102121 product_2 pty_country_41 +2014102121 product_7 pty_country_37 +2014102121 product_9 pty_country_33 +2014102122 product_8 pty_country_34 +2014102122 product_11 pty_country_10 +2014102122 product_9 pty_country_10 +2014102122 product_14 pty_country_10 +2014102122 product_2 pty_country_11 +2014102122 product_8 pty_country_25 +2014102122 product_7 pty_country_25 +2014102122 product_10 pty_country_16 +2014102122 product_6 pty_country_33 +2014102122 product_9 pty_country_27 +2014102123 product_10 pty_country_16 +2014102123 product_13 pty_country_48 +2014102123 product_10 pty_country_34 +2014102123 product_2 pty_country_16 +2014102123 product_7 pty_country_41 +2014102123 product_10 pty_country_26 +2014102123 product_12 pty_country_43 +2014102123 product_10 pty_country_35 +2014102123 product_7 pty_country_11 +2014102123 product_5 pty_country_38 +2014102200 product_12 pty_country_1 +2014102200 product_2 pty_country_24 +2014102200 product_12 pty_country_27 +2014102200 product_1 pty_country_16 +2014102200 product_2 pty_country_42 +2014102200 product_13 pty_country_18 +2014102200 product_8 pty_country_32 +2014102200 product_11 pty_country_25 +2014102200 product_2 pty_country_49 +2014102200 product_3 pty_country_14 +2014102201 product_10 pty_country_16 +2014102201 product_3 pty_country_2 +2014102201 product_9 pty_country_27 +2014102201 product_8 pty_country_6 +2014102201 product_9 pty_country_5 +2014102201 product_13 pty_country_34 +2014102201 product_2 pty_country_50 +2014102201 product_4 pty_country_46 +2014102201 product_14 pty_country_23 +2014102201 product_2 pty_country_23 +2014102202 product_8 pty_country_40 +2014102202 product_8 pty_country_13 +2014102202 product_2 pty_country_9 +2014102202 product_7 pty_country_15 +2014102202 product_7 pty_country_42 +2014102202 product_11 pty_country_17 +2014102202 product_14 pty_country_24 +2014102202 product_5 pty_country_50 +2014102202 product_9 pty_country_1 +2014102202 product_3 pty_country_27 +2014102203 product_10 pty_country_11 +2014102203 product_3 pty_country_19 +2014102203 product_10 pty_country_8 +2014102203 product_14 pty_country_18 +2014102203 product_8 pty_country_11 +2014102203 product_2 pty_country_29 +2014102203 product_2 pty_country_48 +2014102203 product_13 pty_country_23 +2014102203 product_9 pty_country_6 +2014102203 product_1 pty_country_9 +2014102204 product_11 pty_country_2 +2014102204 product_12 pty_country_8 +2014102204 product_3 pty_country_33 +2014102204 product_7 pty_country_25 +2014102204 product_11 pty_country_16 +2014102204 product_3 pty_country_4 +2014102204 product_1 pty_country_24 +2014102204 product_8 pty_country_46 +2014102204 product_7 pty_country_30 +2014102204 product_10 pty_country_2 +2014102205 product_11 pty_country_49 +2014102205 product_6 pty_country_41 +2014102205 product_9 pty_country_9 +2014102205 product_14 pty_country_5 +2014102205 product_10 pty_country_31 +2014102205 product_11 pty_country_20 +2014102205 product_4 pty_country_11 +2014102205 product_3 pty_country_41 +2014102205 product_7 pty_country_15 +2014102205 product_11 pty_country_49 +2014102206 product_5 pty_country_31 +2014102206 product_10 pty_country_13 +2014102206 product_9 pty_country_28 +2014102206 product_1 pty_country_48 +2014102206 product_12 pty_country_31 +2014102206 product_11 pty_country_40 +2014102206 product_11 pty_country_12 +2014102206 product_8 pty_country_16 +2014102206 product_14 pty_country_22 +2014102206 product_7 pty_country_44 +2014102207 product_4 pty_country_1 +2014102207 product_6 pty_country_36 +2014102207 product_1 pty_country_41 +2014102207 product_14 pty_country_16 +2014102207 product_11 pty_country_37 +2014102207 product_14 pty_country_34 +2014102207 product_8 pty_country_42 +2014102207 product_13 pty_country_34 +2014102207 product_2 pty_country_14 +2014102207 product_2 pty_country_18 +2014102208 product_5 pty_country_10 +2014102208 product_4 pty_country_20 +2014102208 product_9 pty_country_32 +2014102208 product_15 pty_country_24 +2014102208 product_7 pty_country_48 +2014102208 product_15 pty_country_36 +2014102208 product_13 pty_country_33 +2014102208 product_12 pty_country_40 +2014102208 product_11 pty_country_40 +2014102208 product_5 pty_country_7 +2014102209 product_3 pty_country_8 +2014102209 product_12 pty_country_10 +2014102209 product_5 pty_country_29 +2014102209 product_7 pty_country_18 +2014102209 product_4 pty_country_31 +2014102209 product_10 pty_country_6 +2014102209 product_8 pty_country_25 +2014102209 product_9 pty_country_2 +2014102209 product_15 pty_country_17 +2014102209 product_14 pty_country_16 +2014102210 product_9 pty_country_8 +2014102210 product_4 pty_country_46 +2014102210 product_5 pty_country_46 +2014102210 product_10 pty_country_38 +2014102210 product_13 pty_country_34 +2014102210 product_8 pty_country_29 +2014102210 product_5 pty_country_28 +2014102210 product_5 pty_country_45 +2014102210 product_2 pty_country_5 +2014102210 product_1 pty_country_48 +2014102211 product_3 pty_country_45 +2014102211 product_13 pty_country_26 +2014102211 product_6 pty_country_15 +2014102211 product_4 pty_country_50 +2014102211 product_11 pty_country_38 +2014102211 product_10 pty_country_29 +2014102211 product_8 pty_country_39 +2014102211 product_12 pty_country_25 +2014102211 product_1 pty_country_25 +2014102211 product_10 pty_country_16 +2014102212 product_8 pty_country_13 +2014102212 product_7 pty_country_48 +2014102212 product_2 pty_country_23 +2014102212 product_14 pty_country_41 +2014102212 product_15 pty_country_27 +2014102212 product_10 pty_country_38 +2014102212 product_9 pty_country_13 +2014102212 product_14 pty_country_40 +2014102212 product_6 pty_country_50 +2014102212 product_8 pty_country_8 +2014102213 product_12 pty_country_30 +2014102213 product_2 pty_country_6 +2014102213 product_15 pty_country_44 +2014102213 product_3 pty_country_35 +2014102213 product_3 pty_country_15 +2014102213 product_1 pty_country_26 +2014102213 product_10 pty_country_33 +2014102213 product_3 pty_country_2 +2014102213 product_9 pty_country_17 +2014102213 product_5 pty_country_15 +2014102214 product_5 pty_country_41 +2014102214 product_5 pty_country_12 +2014102214 product_10 pty_country_27 +2014102214 product_13 pty_country_30 +2014102214 product_10 pty_country_8 +2014102214 product_10 pty_country_43 +2014102214 product_4 pty_country_50 +2014102214 product_8 pty_country_50 +2014102214 product_14 pty_country_11 +2014102214 product_2 pty_country_44 +2014102215 product_6 pty_country_40 +2014102215 product_5 pty_country_2 +2014102215 product_3 pty_country_46 +2014102215 product_7 pty_country_48 +2014102215 product_1 pty_country_34 +2014102215 product_14 pty_country_41 +2014102215 product_10 pty_country_21 +2014102215 product_2 pty_country_13 +2014102215 product_13 pty_country_19 +2014102215 product_7 pty_country_3 +2014102216 product_14 pty_country_42 +2014102216 product_11 pty_country_11 +2014102216 product_14 pty_country_35 +2014102216 product_2 pty_country_21 +2014102216 product_12 pty_country_21 +2014102216 product_6 pty_country_32 +2014102216 product_11 pty_country_27 +2014102216 product_12 pty_country_7 +2014102216 product_9 pty_country_34 +2014102216 product_5 pty_country_7 +2014102217 product_1 pty_country_42 +2014102217 product_1 pty_country_32 +2014102217 product_3 pty_country_1 +2014102217 product_7 pty_country_24 +2014102217 product_13 pty_country_41 +2014102217 product_6 pty_country_9 +2014102217 product_14 pty_country_13 +2014102217 product_15 pty_country_1 +2014102217 product_6 pty_country_41 +2014102217 product_15 pty_country_49 +2014102218 product_14 pty_country_12 +2014102218 product_10 pty_country_26 +2014102218 product_4 pty_country_47 +2014102218 product_1 pty_country_25 +2014102218 product_10 pty_country_50 +2014102218 product_5 pty_country_16 +2014102218 product_13 pty_country_49 +2014102218 product_4 pty_country_14 +2014102218 product_11 pty_country_30 +2014102218 product_4 pty_country_36 +2014102219 product_2 pty_country_23 +2014102219 product_5 pty_country_9 +2014102219 product_2 pty_country_20 +2014102219 product_15 pty_country_45 +2014102219 product_1 pty_country_2 +2014102219 product_13 pty_country_47 +2014102219 product_9 pty_country_42 +2014102219 product_3 pty_country_41 +2014102219 product_14 pty_country_19 +2014102219 product_13 pty_country_13 +2014102220 product_13 pty_country_41 +2014102220 product_6 pty_country_39 +2014102220 product_1 pty_country_40 +2014102220 product_2 pty_country_38 +2014102220 product_6 pty_country_43 +2014102220 product_13 pty_country_12 +2014102220 product_7 pty_country_43 +2014102220 product_14 pty_country_13 +2014102220 product_2 pty_country_2 +2014102220 product_11 pty_country_2 +2014102221 product_10 pty_country_38 +2014102221 product_4 pty_country_1 +2014102221 product_5 pty_country_16 +2014102221 product_6 pty_country_47 +2014102221 product_8 pty_country_20 +2014102221 product_4 pty_country_7 +2014102221 product_1 pty_country_19 +2014102221 product_2 pty_country_45 +2014102221 product_15 pty_country_24 +2014102221 product_13 pty_country_28 +2014102222 product_1 pty_country_46 +2014102222 product_15 pty_country_31 +2014102222 product_6 pty_country_30 +2014102222 product_10 pty_country_46 +2014102222 product_9 pty_country_34 +2014102222 product_14 pty_country_23 +2014102222 product_2 pty_country_23 +2014102222 product_3 pty_country_17 +2014102222 product_2 pty_country_17 +2014102222 product_4 pty_country_21 +2014102223 product_12 pty_country_12 +2014102223 product_1 pty_country_46 +2014102223 product_1 pty_country_18 +2014102223 product_5 pty_country_35 +2014102223 product_12 pty_country_2 +2014102223 product_7 pty_country_14 +2014102223 product_14 pty_country_44 +2014102223 product_12 pty_country_9 +2014102223 product_15 pty_country_1 +2014102223 product_5 pty_country_48 +2014102300 product_8 pty_country_33 +2014102300 product_13 pty_country_13 +2014102300 product_11 pty_country_19 +2014102300 product_6 pty_country_16 +2014102300 product_5 pty_country_11 +2014102300 product_4 pty_country_42 +2014102300 product_10 pty_country_5 +2014102300 product_4 pty_country_5 +2014102300 product_8 pty_country_2 +2014102300 product_13 pty_country_2 +2014102301 product_4 pty_country_24 +2014102301 product_14 pty_country_6 +2014102301 product_12 pty_country_42 +2014102301 product_3 pty_country_7 +2014102301 product_11 pty_country_1 +2014102301 product_7 pty_country_18 +2014102301 product_5 pty_country_29 +2014102301 product_15 pty_country_32 +2014102301 product_6 pty_country_8 +2014102301 product_1 pty_country_44 +2014102302 product_15 pty_country_39 +2014102302 product_12 pty_country_8 +2014102302 product_8 pty_country_47 +2014102302 product_12 pty_country_8 +2014102302 product_3 pty_country_27 +2014102302 product_12 pty_country_27 +2014102302 product_2 pty_country_22 +2014102302 product_4 pty_country_18 +2014102302 product_3 pty_country_34 +2014102302 product_2 pty_country_17 +2014102303 product_12 pty_country_20 +2014102303 product_11 pty_country_7 +2014102303 product_10 pty_country_23 +2014102303 product_13 pty_country_8 +2014102303 product_11 pty_country_19 +2014102303 product_7 pty_country_46 +2014102303 product_5 pty_country_16 +2014102303 product_14 pty_country_5 +2014102303 product_11 pty_country_35 +2014102303 product_12 pty_country_13 +2014102304 product_7 pty_country_15 +2014102304 product_8 pty_country_34 +2014102304 product_3 pty_country_19 +2014102304 product_15 pty_country_45 +2014102304 product_10 pty_country_44 +2014102304 product_10 pty_country_4 +2014102304 product_8 pty_country_48 +2014102304 product_9 pty_country_46 +2014102304 product_9 pty_country_37 +2014102304 product_6 pty_country_1 +2014102305 product_6 pty_country_46 +2014102305 product_15 pty_country_27 +2014102305 product_8 pty_country_35 +2014102305 product_13 pty_country_37 +2014102305 product_7 pty_country_26 +2014102305 product_2 pty_country_41 +2014102305 product_7 pty_country_44 +2014102305 product_9 pty_country_11 +2014102305 product_12 pty_country_9 +2014102305 product_13 pty_country_19 +2014102306 product_5 pty_country_8 +2014102306 product_2 pty_country_25 +2014102306 product_15 pty_country_12 +2014102306 product_2 pty_country_27 +2014102306 product_13 pty_country_2 +2014102306 product_5 pty_country_47 +2014102306 product_14 pty_country_22 +2014102306 product_2 pty_country_31 +2014102306 product_13 pty_country_40 +2014102306 product_1 pty_country_32 +2014102307 product_9 pty_country_44 +2014102307 product_4 pty_country_50 +2014102307 product_8 pty_country_20 +2014102307 product_4 pty_country_11 +2014102307 product_12 pty_country_41 +2014102307 product_3 pty_country_38 +2014102307 product_14 pty_country_2 +2014102307 product_4 pty_country_21 +2014102307 product_6 pty_country_34 +2014102307 product_12 pty_country_14 +2014102308 product_1 pty_country_41 +2014102308 product_5 pty_country_41 +2014102308 product_9 pty_country_10 +2014102308 product_13 pty_country_17 +2014102308 product_8 pty_country_16 +2014102308 product_15 pty_country_31 +2014102308 product_15 pty_country_8 +2014102308 product_6 pty_country_24 +2014102308 product_5 pty_country_37 +2014102308 product_9 pty_country_7 +2014102309 product_2 pty_country_21 +2014102309 product_3 pty_country_16 +2014102309 product_10 pty_country_38 +2014102309 product_10 pty_country_35 +2014102309 product_11 pty_country_7 +2014102309 product_1 pty_country_8 +2014102309 product_3 pty_country_26 +2014102309 product_12 pty_country_26 +2014102309 product_10 pty_country_30 +2014102309 product_8 pty_country_27 +2014102310 product_8 pty_country_9 +2014102310 product_15 pty_country_29 +2014102310 product_8 pty_country_39 +2014102310 product_6 pty_country_4 +2014102310 product_14 pty_country_47 +2014102310 product_6 pty_country_41 +2014102310 product_4 pty_country_20 +2014102310 product_14 pty_country_16 +2014102310 product_14 pty_country_49 +2014102310 product_13 pty_country_38 +2014102311 product_10 pty_country_43 +2014102311 product_12 pty_country_1 +2014102311 product_10 pty_country_31 +2014102311 product_2 pty_country_5 +2014102311 product_10 pty_country_12 +2014102311 product_13 pty_country_33 +2014102311 product_12 pty_country_9 +2014102311 product_1 pty_country_38 +2014102311 product_15 pty_country_17 +2014102311 product_13 pty_country_7 +2014102312 product_1 pty_country_44 +2014102312 product_8 pty_country_28 +2014102312 product_12 pty_country_28 +2014102312 product_9 pty_country_36 +2014102312 product_11 pty_country_35 +2014102312 product_2 pty_country_42 +2014102312 product_2 pty_country_40 +2014102312 product_2 pty_country_23 +2014102312 product_5 pty_country_24 +2014102312 product_13 pty_country_8 +2014102313 product_2 pty_country_7 +2014102313 product_11 pty_country_45 +2014102313 product_11 pty_country_48 +2014102313 product_1 pty_country_28 +2014102313 product_10 pty_country_28 +2014102313 product_1 pty_country_20 +2014102313 product_9 pty_country_44 +2014102313 product_7 pty_country_45 +2014102313 product_6 pty_country_22 +2014102313 product_10 pty_country_33 +2014102314 product_13 pty_country_9 +2014102314 product_15 pty_country_27 +2014102314 product_8 pty_country_48 +2014102314 product_2 pty_country_23 +2014102314 product_9 pty_country_38 +2014102314 product_9 pty_country_29 +2014102314 product_4 pty_country_30 +2014102314 product_2 pty_country_38 +2014102314 product_7 pty_country_26 +2014102314 product_2 pty_country_41 +2014102315 product_1 pty_country_19 +2014102315 product_11 pty_country_5 +2014102315 product_4 pty_country_15 +2014102315 product_7 pty_country_32 +2014102315 product_12 pty_country_44 +2014102315 product_5 pty_country_9 +2014102315 product_10 pty_country_27 +2014102315 product_3 pty_country_46 +2014102315 product_1 pty_country_18 +2014102315 product_12 pty_country_7 +2014102316 product_4 pty_country_10 +2014102316 product_7 pty_country_3 +2014102316 product_3 pty_country_26 +2014102316 product_14 pty_country_25 +2014102316 product_1 pty_country_45 +2014102316 product_14 pty_country_5 +2014102316 product_15 pty_country_29 +2014102316 product_8 pty_country_46 +2014102316 product_2 pty_country_17 +2014102316 product_15 pty_country_30 +2014102317 product_7 pty_country_31 +2014102317 product_13 pty_country_34 +2014102317 product_13 pty_country_16 +2014102317 product_14 pty_country_22 +2014102317 product_12 pty_country_14 +2014102317 product_2 pty_country_2 +2014102317 product_13 pty_country_16 +2014102317 product_11 pty_country_43 +2014102317 product_14 pty_country_9 +2014102317 product_8 pty_country_26 +2014102318 product_2 pty_country_30 +2014102318 product_2 pty_country_49 +2014102318 product_8 pty_country_21 +2014102318 product_15 pty_country_47 +2014102318 product_1 pty_country_4 +2014102318 product_7 pty_country_19 +2014102318 product_7 pty_country_30 +2014102318 product_3 pty_country_37 +2014102318 product_4 pty_country_12 +2014102318 product_13 pty_country_12 +2014102319 product_8 pty_country_9 +2014102319 product_12 pty_country_37 +2014102319 product_4 pty_country_43 +2014102319 product_10 pty_country_12 +2014102319 product_7 pty_country_20 +2014102319 product_15 pty_country_4 +2014102319 product_4 pty_country_22 +2014102319 product_2 pty_country_32 +2014102319 product_14 pty_country_33 +2014102319 product_12 pty_country_11 +2014102320 product_4 pty_country_29 +2014102320 product_3 pty_country_8 +2014102320 product_15 pty_country_27 +2014102320 product_10 pty_country_40 +2014102320 product_13 pty_country_7 +2014102320 product_9 pty_country_49 +2014102320 product_13 pty_country_38 +2014102320 product_3 pty_country_35 +2014102320 product_9 pty_country_38 +2014102320 product_6 pty_country_18 +2014102321 product_8 pty_country_1 +2014102321 product_13 pty_country_1 +2014102321 product_3 pty_country_30 +2014102321 product_4 pty_country_21 +2014102321 product_11 pty_country_25 +2014102321 product_13 pty_country_20 +2014102321 product_4 pty_country_2 +2014102321 product_14 pty_country_3 +2014102321 product_8 pty_country_5 +2014102321 product_5 pty_country_28 +2014102322 product_7 pty_country_29 +2014102322 product_8 pty_country_29 +2014102322 product_7 pty_country_40 +2014102322 product_8 pty_country_14 +2014102322 product_6 pty_country_26 +2014102322 product_1 pty_country_10 +2014102322 product_4 pty_country_19 +2014102322 product_15 pty_country_20 +2014102322 product_5 pty_country_44 +2014102322 product_6 pty_country_34 +2014102323 product_10 pty_country_21 +2014102323 product_6 pty_country_7 +2014102323 product_2 pty_country_40 +2014102323 product_6 pty_country_22 +2014102323 product_5 pty_country_45 +2014102323 product_1 pty_country_24 +2014102323 product_13 pty_country_49 +2014102323 product_1 pty_country_9 +2014102323 product_8 pty_country_18 +2014102323 product_12 pty_country_41 +2014102400 product_5 pty_country_41 +2014102400 product_13 pty_country_19 +2014102400 product_14 pty_country_48 +2014102400 product_12 pty_country_33 +2014102400 product_2 pty_country_27 +2014102400 product_5 pty_country_29 +2014102400 product_11 pty_country_44 +2014102400 product_6 pty_country_24 +2014102400 product_2 pty_country_18 +2014102400 product_1 pty_country_34 +2014102401 product_3 pty_country_32 +2014102401 product_3 pty_country_15 +2014102401 product_3 pty_country_17 +2014102401 product_15 pty_country_12 +2014102401 product_8 pty_country_17 +2014102401 product_3 pty_country_32 +2014102401 product_12 pty_country_21 +2014102401 product_14 pty_country_12 +2014102401 product_6 pty_country_2 +2014102401 product_1 pty_country_17 +2014102402 product_1 pty_country_47 +2014102402 product_5 pty_country_26 +2014102402 product_10 pty_country_35 +2014102402 product_13 pty_country_43 +2014102402 product_1 pty_country_5 +2014102402 product_12 pty_country_30 +2014102402 product_6 pty_country_36 +2014102402 product_5 pty_country_5 +2014102402 product_11 pty_country_9 +2014102402 product_2 pty_country_48 +2014102403 product_15 pty_country_30 +2014102403 product_3 pty_country_28 +2014102403 product_15 pty_country_15 +2014102403 product_9 pty_country_7 +2014102403 product_12 pty_country_42 +2014102403 product_13 pty_country_46 +2014102403 product_9 pty_country_7 +2014102403 product_7 pty_country_18 +2014102403 product_2 pty_country_11 +2014102403 product_14 pty_country_35 +2014102404 product_15 pty_country_46 +2014102404 product_10 pty_country_40 +2014102404 product_7 pty_country_23 +2014102404 product_15 pty_country_49 +2014102404 product_6 pty_country_48 +2014102404 product_15 pty_country_6 +2014102404 product_8 pty_country_31 +2014102404 product_11 pty_country_21 +2014102404 product_9 pty_country_34 +2014102404 product_15 pty_country_9 +2014102405 product_13 pty_country_45 +2014102405 product_8 pty_country_39 +2014102405 product_3 pty_country_7 +2014102405 product_5 pty_country_12 +2014102405 product_8 pty_country_18 +2014102405 product_14 pty_country_11 +2014102405 product_3 pty_country_38 +2014102405 product_8 pty_country_48 +2014102405 product_10 pty_country_46 +2014102405 product_15 pty_country_9 +2014102406 product_14 pty_country_3 +2014102406 product_6 pty_country_27 +2014102406 product_4 pty_country_33 +2014102406 product_11 pty_country_27 +2014102406 product_13 pty_country_4 +2014102406 product_10 pty_country_15 +2014102406 product_6 pty_country_25 +2014102406 product_8 pty_country_12 +2014102406 product_1 pty_country_11 +2014102406 product_4 pty_country_22 +2014102407 product_2 pty_country_12 +2014102407 product_2 pty_country_12 +2014102407 product_7 pty_country_39 +2014102407 product_4 pty_country_28 +2014102407 product_15 pty_country_45 +2014102407 product_14 pty_country_24 +2014102407 product_9 pty_country_16 +2014102407 product_15 pty_country_40 +2014102407 product_12 pty_country_9 +2014102407 product_2 pty_country_37 +2014102408 product_8 pty_country_40 +2014102408 product_3 pty_country_13 +2014102408 product_15 pty_country_17 +2014102408 product_8 pty_country_40 +2014102408 product_15 pty_country_45 +2014102408 product_13 pty_country_37 +2014102408 product_11 pty_country_38 +2014102408 product_15 pty_country_40 +2014102408 product_5 pty_country_24 +2014102408 product_11 pty_country_23 +2014102409 product_6 pty_country_15 +2014102409 product_9 pty_country_8 +2014102409 product_6 pty_country_49 +2014102409 product_5 pty_country_11 +2014102409 product_1 pty_country_37 +2014102409 product_13 pty_country_27 +2014102409 product_9 pty_country_17 +2014102409 product_6 pty_country_47 +2014102409 product_10 pty_country_37 +2014102409 product_14 pty_country_34 +2014102410 product_10 pty_country_46 +2014102410 product_14 pty_country_40 +2014102410 product_14 pty_country_20 +2014102410 product_1 pty_country_42 +2014102410 product_13 pty_country_9 +2014102410 product_1 pty_country_20 +2014102410 product_10 pty_country_22 +2014102410 product_7 pty_country_30 +2014102410 product_15 pty_country_2 +2014102410 product_8 pty_country_8 +2014102411 product_9 pty_country_11 +2014102411 product_5 pty_country_10 +2014102411 product_12 pty_country_3 +2014102411 product_4 pty_country_4 +2014102411 product_3 pty_country_41 +2014102411 product_4 pty_country_8 +2014102411 product_9 pty_country_30 +2014102411 product_11 pty_country_2 +2014102411 product_5 pty_country_5 +2014102411 product_4 pty_country_7 +2014102412 product_5 pty_country_47 +2014102412 product_8 pty_country_34 +2014102412 product_15 pty_country_3 +2014102412 product_1 pty_country_11 +2014102412 product_13 pty_country_16 +2014102412 product_8 pty_country_26 +2014102412 product_3 pty_country_34 +2014102412 product_2 pty_country_16 +2014102412 product_11 pty_country_23 +2014102412 product_14 pty_country_48 +2014102413 product_2 pty_country_25 +2014102413 product_14 pty_country_13 +2014102413 product_13 pty_country_5 +2014102413 product_5 pty_country_26 +2014102413 product_10 pty_country_4 +2014102413 product_15 pty_country_7 +2014102413 product_4 pty_country_21 +2014102413 product_6 pty_country_25 +2014102413 product_6 pty_country_47 +2014102413 product_10 pty_country_50 +2014102414 product_11 pty_country_47 +2014102414 product_12 pty_country_19 +2014102414 product_7 pty_country_9 +2014102414 product_13 pty_country_12 +2014102414 product_3 pty_country_10 +2014102414 product_10 pty_country_15 +2014102414 product_11 pty_country_3 +2014102414 product_5 pty_country_2 +2014102414 product_5 pty_country_11 +2014102414 product_7 pty_country_25 +2014102415 product_9 pty_country_31 +2014102415 product_10 pty_country_21 +2014102415 product_3 pty_country_28 +2014102415 product_6 pty_country_41 +2014102415 product_7 pty_country_17 +2014102415 product_10 pty_country_23 +2014102415 product_14 pty_country_18 +2014102415 product_12 pty_country_3 +2014102415 product_4 pty_country_20 +2014102415 product_3 pty_country_16 +2014102416 product_10 pty_country_16 +2014102416 product_5 pty_country_19 +2014102416 product_2 pty_country_40 +2014102416 product_1 pty_country_13 +2014102416 product_8 pty_country_33 +2014102416 product_13 pty_country_29 +2014102416 product_4 pty_country_46 +2014102416 product_7 pty_country_12 +2014102416 product_11 pty_country_39 +2014102416 product_6 pty_country_16 +2014102417 product_15 pty_country_21 +2014102417 product_5 pty_country_39 +2014102417 product_5 pty_country_28 +2014102417 product_11 pty_country_29 +2014102417 product_3 pty_country_49 +2014102417 product_15 pty_country_19 +2014102417 product_8 pty_country_10 +2014102417 product_13 pty_country_8 +2014102417 product_14 pty_country_15 +2014102417 product_2 pty_country_42 +2014102418 product_8 pty_country_30 +2014102418 product_9 pty_country_23 +2014102418 product_8 pty_country_50 +2014102418 product_6 pty_country_6 +2014102418 product_15 pty_country_45 +2014102418 product_9 pty_country_10 +2014102418 product_14 pty_country_48 +2014102418 product_7 pty_country_33 +2014102418 product_8 pty_country_30 +2014102418 product_7 pty_country_13 +2014102419 product_11 pty_country_50 +2014102419 product_2 pty_country_12 +2014102419 product_12 pty_country_38 +2014102419 product_3 pty_country_22 +2014102419 product_9 pty_country_17 +2014102419 product_6 pty_country_19 +2014102419 product_12 pty_country_2 +2014102419 product_5 pty_country_23 +2014102419 product_14 pty_country_32 +2014102419 product_14 pty_country_15 +2014102420 product_15 pty_country_4 +2014102420 product_9 pty_country_9 +2014102420 product_3 pty_country_25 +2014102420 product_12 pty_country_4 +2014102420 product_6 pty_country_46 +2014102420 product_13 pty_country_9 +2014102420 product_12 pty_country_32 +2014102420 product_4 pty_country_1 +2014102420 product_3 pty_country_27 +2014102420 product_14 pty_country_11 +2014102421 product_8 pty_country_3 +2014102421 product_1 pty_country_25 +2014102421 product_1 pty_country_45 +2014102421 product_5 pty_country_3 +2014102421 product_1 pty_country_1 +2014102421 product_3 pty_country_40 +2014102421 product_13 pty_country_13 +2014102421 product_3 pty_country_13 +2014102421 product_10 pty_country_13 +2014102421 product_5 pty_country_28 +2014102422 product_2 pty_country_23 +2014102422 product_12 pty_country_9 +2014102422 product_14 pty_country_31 +2014102422 product_5 pty_country_43 +2014102422 product_13 pty_country_30 +2014102422 product_15 pty_country_6 +2014102422 product_1 pty_country_10 +2014102422 product_12 pty_country_21 +2014102422 product_3 pty_country_37 +2014102422 product_11 pty_country_46 +2014102423 product_7 pty_country_40 +2014102423 product_9 pty_country_30 +2014102423 product_9 pty_country_8 +2014102423 product_2 pty_country_3 +2014102423 product_15 pty_country_23 +2014102423 product_9 pty_country_4 +2014102423 product_7 pty_country_50 +2014102423 product_1 pty_country_9 +2014102423 product_8 pty_country_7 +2014102423 product_14 pty_country_41 +2014102500 product_7 pty_country_34 +2014102500 product_7 pty_country_43 +2014102500 product_15 pty_country_49 +2014102500 product_14 pty_country_46 +2014102500 product_11 pty_country_34 +2014102500 product_3 pty_country_29 +2014102500 product_2 pty_country_39 +2014102500 product_12 pty_country_25 +2014102500 product_8 pty_country_49 +2014102500 product_5 pty_country_38 +2014102501 product_4 pty_country_25 +2014102501 product_2 pty_country_8 +2014102501 product_9 pty_country_8 +2014102501 product_2 pty_country_37 +2014102501 product_13 pty_country_2 +2014102501 product_13 pty_country_9 +2014102501 product_8 pty_country_40 +2014102501 product_6 pty_country_49 +2014102501 product_14 pty_country_40 +2014102501 product_3 pty_country_45 +2014102502 product_12 pty_country_35 +2014102502 product_13 pty_country_12 +2014102502 product_11 pty_country_2 +2014102502 product_4 pty_country_42 +2014102502 product_11 pty_country_42 +2014102502 product_15 pty_country_2 +2014102502 product_11 pty_country_46 +2014102502 product_11 pty_country_19 +2014102502 product_3 pty_country_33 +2014102502 product_3 pty_country_6 +2014102503 product_6 pty_country_29 +2014102503 product_1 pty_country_48 +2014102503 product_14 pty_country_36 +2014102503 product_3 pty_country_7 +2014102503 product_13 pty_country_13 +2014102503 product_12 pty_country_42 +2014102503 product_8 pty_country_10 +2014102503 product_15 pty_country_40 +2014102503 product_5 pty_country_7 +2014102503 product_10 pty_country_31 +2014102504 product_4 pty_country_47 +2014102504 product_3 pty_country_8 +2014102504 product_12 pty_country_39 +2014102504 product_4 pty_country_2 +2014102504 product_11 pty_country_41 +2014102504 product_13 pty_country_44 +2014102504 product_10 pty_country_38 +2014102504 product_1 pty_country_7 +2014102504 product_6 pty_country_3 +2014102504 product_7 pty_country_23 +2014102505 product_9 pty_country_8 +2014102505 product_7 pty_country_24 +2014102505 product_6 pty_country_30 +2014102505 product_4 pty_country_43 +2014102505 product_4 pty_country_11 +2014102505 product_1 pty_country_11 +2014102505 product_8 pty_country_48 +2014102505 product_15 pty_country_26 +2014102505 product_1 pty_country_32 +2014102505 product_8 pty_country_1 +2014102506 product_4 pty_country_28 +2014102506 product_10 pty_country_21 +2014102506 product_7 pty_country_22 +2014102506 product_12 pty_country_28 +2014102506 product_2 pty_country_41 +2014102506 product_13 pty_country_31 +2014102506 product_1 pty_country_48 +2014102506 product_2 pty_country_49 +2014102506 product_14 pty_country_34 +2014102506 product_2 pty_country_23 +2014102507 product_10 pty_country_32 +2014102507 product_12 pty_country_31 +2014102507 product_9 pty_country_12 +2014102507 product_6 pty_country_45 +2014102507 product_15 pty_country_36 +2014102507 product_6 pty_country_44 +2014102507 product_11 pty_country_18 +2014102507 product_6 pty_country_37 +2014102507 product_5 pty_country_32 +2014102507 product_3 pty_country_37 +2014102508 product_12 pty_country_20 +2014102508 product_9 pty_country_42 +2014102508 product_10 pty_country_49 +2014102508 product_14 pty_country_38 +2014102508 product_4 pty_country_39 +2014102508 product_12 pty_country_10 +2014102508 product_15 pty_country_4 +2014102508 product_8 pty_country_9 +2014102508 product_8 pty_country_40 +2014102508 product_7 pty_country_48 +2014102509 product_8 pty_country_41 +2014102509 product_5 pty_country_17 +2014102509 product_11 pty_country_50 +2014102509 product_15 pty_country_38 +2014102509 product_9 pty_country_31 +2014102509 product_12 pty_country_8 +2014102509 product_10 pty_country_10 +2014102509 product_2 pty_country_5 +2014102509 product_8 pty_country_40 +2014102509 product_3 pty_country_39 +2014102510 product_10 pty_country_48 +2014102510 product_1 pty_country_37 +2014102510 product_6 pty_country_20 +2014102510 product_9 pty_country_40 +2014102510 product_10 pty_country_25 +2014102510 product_4 pty_country_47 +2014102510 product_12 pty_country_4 +2014102510 product_11 pty_country_27 +2014102510 product_15 pty_country_18 +2014102510 product_3 pty_country_39 +2014102511 product_6 pty_country_22 +2014102511 product_8 pty_country_2 +2014102511 product_8 pty_country_39 +2014102511 product_4 pty_country_33 +2014102511 product_9 pty_country_4 +2014102511 product_10 pty_country_27 +2014102511 product_3 pty_country_25 +2014102511 product_11 pty_country_20 +2014102511 product_12 pty_country_13 +2014102511 product_2 pty_country_30 +2014102512 product_11 pty_country_18 +2014102512 product_3 pty_country_1 +2014102512 product_14 pty_country_38 +2014102512 product_7 pty_country_19 +2014102512 product_14 pty_country_41 +2014102512 product_14 pty_country_47 +2014102512 product_2 pty_country_5 +2014102512 product_5 pty_country_21 +2014102512 product_1 pty_country_35 +2014102512 product_12 pty_country_3 +2014102513 product_13 pty_country_8 +2014102513 product_5 pty_country_33 +2014102513 product_7 pty_country_18 +2014102513 product_12 pty_country_27 +2014102513 product_7 pty_country_31 +2014102513 product_12 pty_country_48 +2014102513 product_8 pty_country_22 +2014102513 product_7 pty_country_44 +2014102513 product_3 pty_country_8 +2014102513 product_8 pty_country_5 +2014102514 product_12 pty_country_3 +2014102514 product_12 pty_country_41 +2014102514 product_12 pty_country_11 +2014102514 product_15 pty_country_17 +2014102514 product_3 pty_country_17 +2014102514 product_1 pty_country_46 +2014102514 product_3 pty_country_9 +2014102514 product_14 pty_country_1 +2014102514 product_1 pty_country_32 +2014102514 product_15 pty_country_40 +2014102515 product_6 pty_country_38 +2014102515 product_4 pty_country_44 +2014102515 product_8 pty_country_35 +2014102515 product_15 pty_country_31 +2014102515 product_10 pty_country_34 +2014102515 product_4 pty_country_1 +2014102515 product_8 pty_country_37 +2014102515 product_10 pty_country_2 +2014102515 product_14 pty_country_29 +2014102515 product_1 pty_country_20 +2014102516 product_13 pty_country_24 +2014102516 product_10 pty_country_47 +2014102516 product_14 pty_country_12 +2014102516 product_8 pty_country_36 +2014102516 product_4 pty_country_13 +2014102516 product_13 pty_country_45 +2014102516 product_14 pty_country_14 +2014102516 product_11 pty_country_5 +2014102516 product_10 pty_country_23 +2014102516 product_13 pty_country_45 +2014102517 product_7 pty_country_2 +2014102517 product_3 pty_country_21 +2014102517 product_11 pty_country_3 +2014102517 product_13 pty_country_1 +2014102517 product_8 pty_country_13 +2014102517 product_3 pty_country_21 +2014102517 product_11 pty_country_14 +2014102517 product_9 pty_country_7 +2014102517 product_10 pty_country_8 +2014102517 product_5 pty_country_48 +2014102518 product_2 pty_country_50 +2014102518 product_4 pty_country_20 +2014102518 product_7 pty_country_39 +2014102518 product_9 pty_country_33 +2014102518 product_2 pty_country_26 +2014102518 product_11 pty_country_47 +2014102518 product_9 pty_country_32 +2014102518 product_4 pty_country_41 +2014102518 product_14 pty_country_2 +2014102518 product_8 pty_country_48 +2014102519 product_8 pty_country_12 +2014102519 product_11 pty_country_18 +2014102519 product_5 pty_country_36 +2014102519 product_11 pty_country_32 +2014102519 product_11 pty_country_50 +2014102519 product_14 pty_country_19 +2014102519 product_7 pty_country_15 +2014102519 product_2 pty_country_42 +2014102519 product_4 pty_country_49 +2014102519 product_13 pty_country_12 +2014102520 product_6 pty_country_8 +2014102520 product_7 pty_country_27 +2014102520 product_9 pty_country_2 +2014102520 product_1 pty_country_15 +2014102520 product_4 pty_country_7 +2014102520 product_9 pty_country_6 +2014102520 product_12 pty_country_38 +2014102520 product_6 pty_country_29 +2014102520 product_14 pty_country_24 +2014102520 product_10 pty_country_2 +2014102521 product_11 pty_country_8 +2014102521 product_9 pty_country_39 +2014102521 product_3 pty_country_50 +2014102521 product_10 pty_country_25 +2014102521 product_10 pty_country_16 +2014102521 product_13 pty_country_39 +2014102521 product_2 pty_country_11 +2014102521 product_13 pty_country_33 +2014102521 product_2 pty_country_7 +2014102521 product_2 pty_country_25 +2014102522 product_11 pty_country_48 +2014102522 product_10 pty_country_9 +2014102522 product_10 pty_country_15 +2014102522 product_7 pty_country_28 +2014102522 product_4 pty_country_45 +2014102522 product_1 pty_country_17 +2014102522 product_10 pty_country_5 +2014102522 product_10 pty_country_9 +2014102522 product_4 pty_country_40 +2014102522 product_15 pty_country_37 +2014102523 product_8 pty_country_32 +2014102523 product_4 pty_country_30 +2014102523 product_13 pty_country_7 +2014102523 product_12 pty_country_5 +2014102523 product_12 pty_country_23 +2014102523 product_3 pty_country_7 +2014102523 product_9 pty_country_27 +2014102523 product_2 pty_country_8 +2014102523 product_15 pty_country_4 +2014102523 product_3 pty_country_12 +2014102600 product_3 pty_country_22 +2014102600 product_2 pty_country_10 +2014102600 product_9 pty_country_20 +2014102600 product_1 pty_country_28 +2014102600 product_7 pty_country_13 +2014102600 product_3 pty_country_26 +2014102600 product_11 pty_country_47 +2014102600 product_2 pty_country_1 +2014102600 product_12 pty_country_23 +2014102600 product_13 pty_country_46 +2014102601 product_10 pty_country_12 +2014102601 product_1 pty_country_47 +2014102601 product_6 pty_country_47 +2014102601 product_7 pty_country_27 +2014102601 product_4 pty_country_3 +2014102601 product_9 pty_country_27 +2014102601 product_8 pty_country_34 +2014102601 product_8 pty_country_19 +2014102601 product_13 pty_country_7 +2014102601 product_12 pty_country_40 +2014102602 product_1 pty_country_21 +2014102602 product_4 pty_country_42 +2014102602 product_15 pty_country_22 +2014102602 product_12 pty_country_13 +2014102602 product_7 pty_country_39 +2014102602 product_14 pty_country_30 +2014102602 product_10 pty_country_29 +2014102602 product_7 pty_country_18 +2014102602 product_9 pty_country_23 +2014102602 product_12 pty_country_49 +2014102603 product_13 pty_country_27 +2014102603 product_9 pty_country_2 +2014102603 product_5 pty_country_32 +2014102603 product_12 pty_country_8 +2014102603 product_11 pty_country_16 +2014102603 product_4 pty_country_39 +2014102603 product_3 pty_country_35 +2014102603 product_8 pty_country_36 +2014102603 product_3 pty_country_15 +2014102603 product_1 pty_country_32 +2014102604 product_13 pty_country_28 +2014102604 product_8 pty_country_36 +2014102604 product_15 pty_country_16 +2014102604 product_14 pty_country_15 +2014102604 product_12 pty_country_30 +2014102604 product_9 pty_country_40 +2014102604 product_6 pty_country_19 +2014102604 product_7 pty_country_41 +2014102604 product_5 pty_country_42 +2014102604 product_3 pty_country_34 +2014102605 product_9 pty_country_41 +2014102605 product_12 pty_country_29 +2014102605 product_5 pty_country_10 +2014102605 product_13 pty_country_41 +2014102605 product_5 pty_country_5 +2014102605 product_2 pty_country_41 +2014102605 product_5 pty_country_41 +2014102605 product_6 pty_country_25 +2014102605 product_11 pty_country_20 +2014102605 product_15 pty_country_41 +2014102606 product_9 pty_country_13 +2014102606 product_15 pty_country_20 +2014102606 product_7 pty_country_6 +2014102606 product_11 pty_country_13 +2014102606 product_1 pty_country_24 +2014102606 product_15 pty_country_3 +2014102606 product_8 pty_country_48 +2014102606 product_10 pty_country_17 +2014102606 product_1 pty_country_12 +2014102606 product_5 pty_country_7 +2014102607 product_8 pty_country_2 +2014102607 product_14 pty_country_2 +2014102607 product_5 pty_country_15 +2014102607 product_5 pty_country_48 +2014102607 product_11 pty_country_11 +2014102607 product_14 pty_country_5 +2014102607 product_13 pty_country_5 +2014102607 product_15 pty_country_25 +2014102607 product_2 pty_country_38 +2014102607 product_14 pty_country_25 +2014102608 product_15 pty_country_27 +2014102608 product_4 pty_country_2 +2014102608 product_10 pty_country_31 +2014102608 product_7 pty_country_8 +2014102608 product_3 pty_country_1 +2014102608 product_6 pty_country_49 +2014102608 product_4 pty_country_39 +2014102608 product_9 pty_country_2 +2014102608 product_3 pty_country_6 +2014102608 product_9 pty_country_8 +2014102609 product_11 pty_country_7 +2014102609 product_10 pty_country_18 +2014102609 product_12 pty_country_27 +2014102609 product_8 pty_country_39 +2014102609 product_14 pty_country_11 +2014102609 product_2 pty_country_9 +2014102609 product_10 pty_country_28 +2014102609 product_12 pty_country_18 +2014102609 product_1 pty_country_40 +2014102609 product_2 pty_country_10 +2014102610 product_5 pty_country_9 +2014102610 product_2 pty_country_10 +2014102610 product_9 pty_country_13 +2014102610 product_12 pty_country_35 +2014102610 product_5 pty_country_12 +2014102610 product_13 pty_country_50 +2014102610 product_4 pty_country_37 +2014102610 product_11 pty_country_50 +2014102610 product_13 pty_country_49 +2014102610 product_9 pty_country_44 +2014102611 product_13 pty_country_23 +2014102611 product_11 pty_country_7 +2014102611 product_1 pty_country_44 +2014102611 product_15 pty_country_21 +2014102611 product_12 pty_country_15 +2014102611 product_1 pty_country_8 +2014102611 product_6 pty_country_29 +2014102611 product_7 pty_country_5 +2014102611 product_6 pty_country_48 +2014102611 product_7 pty_country_10 +2014102612 product_11 pty_country_11 +2014102612 product_9 pty_country_10 +2014102612 product_14 pty_country_16 +2014102612 product_15 pty_country_14 +2014102612 product_10 pty_country_17 +2014102612 product_13 pty_country_1 +2014102612 product_2 pty_country_15 +2014102612 product_4 pty_country_6 +2014102612 product_9 pty_country_2 +2014102612 product_9 pty_country_23 +2014102613 product_5 pty_country_47 +2014102613 product_8 pty_country_33 +2014102613 product_5 pty_country_37 +2014102613 product_12 pty_country_38 +2014102613 product_6 pty_country_48 +2014102613 product_14 pty_country_5 +2014102613 product_1 pty_country_45 +2014102613 product_1 pty_country_12 +2014102613 product_4 pty_country_13 +2014102613 product_1 pty_country_44 +2014102614 product_14 pty_country_35 +2014102614 product_8 pty_country_30 +2014102614 product_14 pty_country_30 +2014102614 product_2 pty_country_22 +2014102614 product_6 pty_country_9 +2014102614 product_12 pty_country_48 +2014102614 product_8 pty_country_34 +2014102614 product_9 pty_country_39 +2014102614 product_10 pty_country_9 +2014102614 product_13 pty_country_22 +2014102615 product_5 pty_country_34 +2014102615 product_13 pty_country_20 +2014102615 product_14 pty_country_34 +2014102615 product_1 pty_country_19 +2014102615 product_2 pty_country_1 +2014102615 product_5 pty_country_24 +2014102615 product_8 pty_country_27 +2014102615 product_13 pty_country_23 +2014102615 product_13 pty_country_39 +2014102615 product_11 pty_country_47 +2014102616 product_4 pty_country_6 +2014102616 product_1 pty_country_35 +2014102616 product_3 pty_country_30 +2014102616 product_14 pty_country_42 +2014102616 product_5 pty_country_45 +2014102616 product_10 pty_country_33 +2014102616 product_9 pty_country_49 +2014102616 product_2 pty_country_7 +2014102616 product_1 pty_country_41 +2014102616 product_2 pty_country_48 +2014102617 product_4 pty_country_46 +2014102617 product_9 pty_country_25 +2014102617 product_6 pty_country_49 +2014102617 product_3 pty_country_32 +2014102617 product_5 pty_country_27 +2014102617 product_10 pty_country_40 +2014102617 product_14 pty_country_16 +2014102617 product_8 pty_country_48 +2014102617 product_8 pty_country_40 +2014102617 product_15 pty_country_39 +2014102618 product_13 pty_country_8 +2014102618 product_10 pty_country_7 +2014102618 product_10 pty_country_6 +2014102618 product_1 pty_country_30 +2014102618 product_4 pty_country_30 +2014102618 product_6 pty_country_42 +2014102618 product_12 pty_country_43 +2014102618 product_6 pty_country_16 +2014102618 product_8 pty_country_4 +2014102618 product_6 pty_country_2 +2014102619 product_15 pty_country_39 +2014102619 product_11 pty_country_22 +2014102619 product_6 pty_country_39 +2014102619 product_7 pty_country_50 +2014102619 product_4 pty_country_28 +2014102619 product_8 pty_country_3 +2014102619 product_3 pty_country_39 +2014102619 product_10 pty_country_4 +2014102619 product_10 pty_country_4 +2014102619 product_3 pty_country_4 +2014102620 product_14 pty_country_31 +2014102620 product_2 pty_country_12 +2014102620 product_1 pty_country_19 +2014102620 product_10 pty_country_4 +2014102620 product_12 pty_country_21 +2014102620 product_2 pty_country_47 +2014102620 product_14 pty_country_46 +2014102620 product_1 pty_country_17 +2014102620 product_14 pty_country_43 +2014102620 product_15 pty_country_12 +2014102621 product_9 pty_country_31 +2014102621 product_13 pty_country_30 +2014102621 product_2 pty_country_39 +2014102621 product_8 pty_country_44 +2014102621 product_9 pty_country_24 +2014102621 product_13 pty_country_11 +2014102621 product_4 pty_country_12 +2014102621 product_4 pty_country_47 +2014102621 product_9 pty_country_19 +2014102621 product_12 pty_country_23 +2014102622 product_5 pty_country_37 +2014102622 product_12 pty_country_13 +2014102622 product_10 pty_country_26 +2014102622 product_5 pty_country_12 +2014102622 product_7 pty_country_16 +2014102622 product_2 pty_country_34 +2014102622 product_9 pty_country_41 +2014102622 product_4 pty_country_25 +2014102622 product_7 pty_country_9 +2014102622 product_2 pty_country_44 +2014102623 product_6 pty_country_14 +2014102623 product_6 pty_country_9 +2014102623 product_13 pty_country_47 +2014102623 product_14 pty_country_36 +2014102623 product_7 pty_country_47 +2014102623 product_7 pty_country_3 +2014102623 product_2 pty_country_23 +2014102623 product_2 pty_country_10 +2014102623 product_13 pty_country_49 +2014102623 product_11 pty_country_18 +2014102700 product_3 pty_country_34 +2014102700 product_4 pty_country_50 +2014102700 product_14 pty_country_15 +2014102700 product_9 pty_country_13 +2014102700 product_11 pty_country_17 +2014102700 product_15 pty_country_42 +2014102700 product_8 pty_country_24 +2014102700 product_12 pty_country_25 +2014102700 product_11 pty_country_24 +2014102700 product_1 pty_country_5 +2014102701 product_8 pty_country_18 +2014102701 product_11 pty_country_33 +2014102701 product_1 pty_country_9 +2014102701 product_9 pty_country_12 +2014102701 product_13 pty_country_19 +2014102701 product_3 pty_country_38 +2014102701 product_13 pty_country_48 +2014102701 product_10 pty_country_6 +2014102701 product_7 pty_country_42 +2014102701 product_14 pty_country_41 +2014102702 product_13 pty_country_12 +2014102702 product_3 pty_country_30 +2014102702 product_9 pty_country_28 +2014102702 product_7 pty_country_21 +2014102702 product_11 pty_country_23 +2014102702 product_4 pty_country_35 +2014102702 product_1 pty_country_7 +2014102702 product_8 pty_country_1 +2014102702 product_2 pty_country_33 +2014102702 product_6 pty_country_25 +2014102703 product_12 pty_country_4 +2014102703 product_3 pty_country_27 +2014102703 product_11 pty_country_2 +2014102703 product_1 pty_country_43 +2014102703 product_12 pty_country_4 +2014102703 product_4 pty_country_4 +2014102703 product_7 pty_country_32 +2014102703 product_9 pty_country_49 +2014102703 product_4 pty_country_13 +2014102703 product_2 pty_country_3 +2014102704 product_1 pty_country_36 +2014102704 product_12 pty_country_12 +2014102704 product_1 pty_country_35 +2014102704 product_3 pty_country_25 +2014102704 product_8 pty_country_33 +2014102704 product_14 pty_country_37 +2014102704 product_15 pty_country_8 +2014102704 product_1 pty_country_25 +2014102704 product_8 pty_country_38 +2014102704 product_11 pty_country_3 +2014102705 product_13 pty_country_8 +2014102705 product_7 pty_country_46 +2014102705 product_10 pty_country_24 +2014102705 product_9 pty_country_16 +2014102705 product_2 pty_country_13 +2014102705 product_4 pty_country_12 +2014102705 product_10 pty_country_38 +2014102705 product_3 pty_country_9 +2014102705 product_3 pty_country_27 +2014102705 product_8 pty_country_28 +2014102706 product_10 pty_country_35 +2014102706 product_11 pty_country_2 +2014102706 product_1 pty_country_7 +2014102706 product_14 pty_country_10 +2014102706 product_11 pty_country_40 +2014102706 product_5 pty_country_15 +2014102706 product_2 pty_country_44 +2014102706 product_2 pty_country_36 +2014102706 product_14 pty_country_21 +2014102706 product_4 pty_country_29 +2014102707 product_14 pty_country_25 +2014102707 product_13 pty_country_17 +2014102707 product_8 pty_country_39 +2014102707 product_10 pty_country_21 +2014102707 product_7 pty_country_50 +2014102707 product_12 pty_country_8 +2014102707 product_3 pty_country_2 +2014102707 product_13 pty_country_41 +2014102707 product_9 pty_country_16 +2014102707 product_14 pty_country_17 +2014102708 product_6 pty_country_1 +2014102708 product_12 pty_country_33 +2014102708 product_8 pty_country_37 +2014102708 product_3 pty_country_50 +2014102708 product_4 pty_country_20 +2014102708 product_14 pty_country_34 +2014102708 product_6 pty_country_13 +2014102708 product_12 pty_country_23 +2014102708 product_11 pty_country_42 +2014102708 product_7 pty_country_35 +2014102709 product_11 pty_country_48 +2014102709 product_12 pty_country_11 +2014102709 product_6 pty_country_17 +2014102709 product_10 pty_country_41 +2014102709 product_8 pty_country_8 +2014102709 product_7 pty_country_41 +2014102709 product_15 pty_country_13 +2014102709 product_10 pty_country_22 +2014102709 product_15 pty_country_5 +2014102709 product_12 pty_country_47 +2014102710 product_11 pty_country_16 +2014102710 product_4 pty_country_21 +2014102710 product_6 pty_country_20 +2014102710 product_8 pty_country_42 +2014102710 product_13 pty_country_9 +2014102710 product_3 pty_country_47 +2014102710 product_4 pty_country_2 +2014102710 product_15 pty_country_33 +2014102710 product_4 pty_country_32 +2014102710 product_15 pty_country_49 +2014102711 product_1 pty_country_37 +2014102711 product_15 pty_country_10 +2014102711 product_7 pty_country_16 +2014102711 product_9 pty_country_45 +2014102711 product_13 pty_country_11 +2014102711 product_3 pty_country_33 +2014102711 product_2 pty_country_33 +2014102711 product_6 pty_country_18 +2014102711 product_9 pty_country_27 +2014102711 product_5 pty_country_31 +2014102712 product_4 pty_country_44 +2014102712 product_8 pty_country_40 +2014102712 product_11 pty_country_21 +2014102712 product_3 pty_country_40 +2014102712 product_9 pty_country_4 +2014102712 product_2 pty_country_31 +2014102712 product_10 pty_country_27 +2014102712 product_8 pty_country_38 +2014102712 product_12 pty_country_1 +2014102712 product_14 pty_country_19 +2014102713 product_8 pty_country_46 +2014102713 product_1 pty_country_11 +2014102713 product_6 pty_country_20 +2014102713 product_6 pty_country_8 +2014102713 product_9 pty_country_45 +2014102713 product_12 pty_country_46 +2014102713 product_3 pty_country_8 +2014102713 product_1 pty_country_49 +2014102713 product_9 pty_country_18 +2014102713 product_14 pty_country_42 +2014102714 product_12 pty_country_12 +2014102714 product_9 pty_country_13 +2014102714 product_5 pty_country_27 +2014102714 product_10 pty_country_2 +2014102714 product_9 pty_country_20 +2014102714 product_15 pty_country_10 +2014102714 product_15 pty_country_49 +2014102714 product_11 pty_country_39 +2014102714 product_8 pty_country_2 +2014102714 product_2 pty_country_36 +2014102715 product_1 pty_country_37 +2014102715 product_11 pty_country_37 +2014102715 product_7 pty_country_16 +2014102715 product_1 pty_country_39 +2014102715 product_1 pty_country_34 +2014102715 product_2 pty_country_25 +2014102715 product_11 pty_country_8 +2014102715 product_15 pty_country_50 +2014102715 product_11 pty_country_39 +2014102715 product_10 pty_country_27 +2014102716 product_14 pty_country_5 +2014102716 product_3 pty_country_31 +2014102716 product_8 pty_country_36 +2014102716 product_9 pty_country_26 +2014102716 product_5 pty_country_31 +2014102716 product_7 pty_country_18 +2014102716 product_11 pty_country_22 +2014102716 product_4 pty_country_4 +2014102716 product_1 pty_country_43 +2014102716 product_4 pty_country_18 +2014102717 product_6 pty_country_12 +2014102717 product_5 pty_country_24 +2014102717 product_7 pty_country_28 +2014102717 product_5 pty_country_45 +2014102717 product_13 pty_country_31 +2014102717 product_12 pty_country_19 +2014102717 product_5 pty_country_20 +2014102717 product_14 pty_country_41 +2014102717 product_3 pty_country_12 +2014102717 product_1 pty_country_41 +2014102718 product_8 pty_country_3 +2014102718 product_11 pty_country_25 +2014102718 product_14 pty_country_21 +2014102718 product_5 pty_country_2 +2014102718 product_4 pty_country_7 +2014102718 product_15 pty_country_11 +2014102718 product_2 pty_country_12 +2014102718 product_4 pty_country_24 +2014102718 product_7 pty_country_25 +2014102718 product_9 pty_country_42 +2014102719 product_15 pty_country_45 +2014102719 product_9 pty_country_44 +2014102719 product_15 pty_country_32 +2014102719 product_8 pty_country_31 +2014102719 product_5 pty_country_27 +2014102719 product_14 pty_country_24 +2014102719 product_13 pty_country_48 +2014102719 product_10 pty_country_6 +2014102719 product_7 pty_country_14 +2014102719 product_1 pty_country_2 +2014102720 product_4 pty_country_49 +2014102720 product_4 pty_country_4 +2014102720 product_14 pty_country_29 +2014102720 product_5 pty_country_35 +2014102720 product_8 pty_country_47 +2014102720 product_4 pty_country_16 +2014102720 product_11 pty_country_18 +2014102720 product_3 pty_country_28 +2014102720 product_10 pty_country_11 +2014102720 product_11 pty_country_26 +2014102721 product_6 pty_country_34 +2014102721 product_11 pty_country_9 +2014102721 product_15 pty_country_10 +2014102721 product_14 pty_country_21 +2014102721 product_3 pty_country_12 +2014102721 product_15 pty_country_36 +2014102721 product_4 pty_country_43 +2014102721 product_15 pty_country_34 +2014102721 product_11 pty_country_8 +2014102721 product_9 pty_country_24 +2014102722 product_12 pty_country_26 +2014102722 product_12 pty_country_20 +2014102722 product_14 pty_country_39 +2014102722 product_5 pty_country_39 +2014102722 product_10 pty_country_4 +2014102722 product_7 pty_country_12 +2014102722 product_4 pty_country_13 +2014102722 product_10 pty_country_41 +2014102722 product_8 pty_country_15 +2014102722 product_3 pty_country_4 +2014102723 product_11 pty_country_25 +2014102723 product_8 pty_country_4 +2014102723 product_10 pty_country_4 +2014102723 product_10 pty_country_36 +2014102723 product_4 pty_country_11 +2014102723 product_8 pty_country_31 +2014102723 product_3 pty_country_47 +2014102723 product_15 pty_country_39 +2014102723 product_3 pty_country_7 +2014102723 product_14 pty_country_37 +2014102800 product_4 pty_country_28 +2014102800 product_2 pty_country_45 +2014102800 product_10 pty_country_28 +2014102800 product_7 pty_country_44 +2014102800 product_9 pty_country_39 +2014102800 product_12 pty_country_20 +2014102800 product_15 pty_country_41 +2014102800 product_12 pty_country_43 +2014102800 product_12 pty_country_45 +2014102800 product_13 pty_country_30 +2014102801 product_3 pty_country_19 +2014102801 product_1 pty_country_5 +2014102801 product_2 pty_country_15 +2014102801 product_1 pty_country_15 +2014102801 product_4 pty_country_25 +2014102801 product_9 pty_country_17 +2014102801 product_12 pty_country_13 +2014102801 product_12 pty_country_22 +2014102801 product_6 pty_country_46 +2014102801 product_9 pty_country_2 +2014102802 product_5 pty_country_13 +2014102802 product_11 pty_country_8 +2014102802 product_8 pty_country_44 +2014102802 product_13 pty_country_34 +2014102802 product_3 pty_country_8 +2014102802 product_13 pty_country_30 +2014102802 product_5 pty_country_8 +2014102802 product_5 pty_country_42 +2014102802 product_11 pty_country_38 +2014102802 product_15 pty_country_28 +2014102803 product_15 pty_country_23 +2014102803 product_3 pty_country_7 +2014102803 product_2 pty_country_33 +2014102803 product_4 pty_country_48 +2014102803 product_9 pty_country_47 +2014102803 product_15 pty_country_41 +2014102803 product_8 pty_country_40 +2014102803 product_5 pty_country_32 +2014102803 product_4 pty_country_26 +2014102803 product_12 pty_country_43 +2014102804 product_3 pty_country_49 +2014102804 product_7 pty_country_39 +2014102804 product_5 pty_country_48 +2014102804 product_4 pty_country_49 +2014102804 product_10 pty_country_8 +2014102804 product_11 pty_country_3 +2014102804 product_14 pty_country_26 +2014102804 product_2 pty_country_30 +2014102804 product_9 pty_country_32 +2014102804 product_15 pty_country_12 +2014102805 product_14 pty_country_12 +2014102805 product_7 pty_country_15 +2014102805 product_15 pty_country_7 +2014102805 product_13 pty_country_32 +2014102805 product_11 pty_country_7 +2014102805 product_15 pty_country_36 +2014102805 product_2 pty_country_25 +2014102805 product_1 pty_country_29 +2014102805 product_9 pty_country_33 +2014102805 product_13 pty_country_40 +2014102806 product_15 pty_country_11 +2014102806 product_3 pty_country_36 +2014102806 product_3 pty_country_35 +2014102806 product_14 pty_country_18 +2014102806 product_5 pty_country_44 +2014102806 product_11 pty_country_23 +2014102806 product_9 pty_country_34 +2014102806 product_14 pty_country_33 +2014102806 product_1 pty_country_31 +2014102806 product_6 pty_country_49 +2014102807 product_3 pty_country_22 +2014102807 product_9 pty_country_16 +2014102807 product_14 pty_country_16 +2014102807 product_7 pty_country_17 +2014102807 product_14 pty_country_24 +2014102807 product_14 pty_country_50 +2014102807 product_10 pty_country_9 +2014102807 product_6 pty_country_36 +2014102807 product_6 pty_country_47 +2014102807 product_5 pty_country_46 +2014102808 product_13 pty_country_2 +2014102808 product_9 pty_country_50 +2014102808 product_10 pty_country_40 +2014102808 product_11 pty_country_21 +2014102808 product_4 pty_country_7 +2014102808 product_4 pty_country_25 +2014102808 product_5 pty_country_1 +2014102808 product_9 pty_country_36 +2014102808 product_14 pty_country_3 +2014102808 product_15 pty_country_25 +2014102809 product_4 pty_country_34 +2014102809 product_14 pty_country_48 +2014102809 product_15 pty_country_6 +2014102809 product_13 pty_country_49 +2014102809 product_13 pty_country_1 +2014102809 product_10 pty_country_3 +2014102809 product_7 pty_country_7 +2014102809 product_5 pty_country_49 +2014102809 product_3 pty_country_12 +2014102809 product_5 pty_country_50 +2014102810 product_1 pty_country_1 +2014102810 product_11 pty_country_5 +2014102810 product_11 pty_country_15 +2014102810 product_1 pty_country_15 +2014102810 product_9 pty_country_30 +2014102810 product_6 pty_country_3 +2014102810 product_1 pty_country_45 +2014102810 product_7 pty_country_42 +2014102810 product_15 pty_country_30 +2014102810 product_5 pty_country_1 +2014102811 product_2 pty_country_25 +2014102811 product_9 pty_country_38 +2014102811 product_11 pty_country_7 +2014102811 product_3 pty_country_45 +2014102811 product_2 pty_country_25 +2014102811 product_13 pty_country_46 +2014102811 product_3 pty_country_18 +2014102811 product_8 pty_country_18 +2014102811 product_9 pty_country_24 +2014102811 product_5 pty_country_31 +2014102812 product_6 pty_country_39 +2014102812 product_6 pty_country_28 +2014102812 product_1 pty_country_14 +2014102812 product_9 pty_country_22 +2014102812 product_5 pty_country_37 +2014102812 product_14 pty_country_48 +2014102812 product_8 pty_country_26 +2014102812 product_1 pty_country_50 +2014102812 product_5 pty_country_22 +2014102812 product_14 pty_country_50 +2014102813 product_1 pty_country_34 +2014102813 product_13 pty_country_19 +2014102813 product_4 pty_country_26 +2014102813 product_13 pty_country_23 +2014102813 product_1 pty_country_17 +2014102813 product_11 pty_country_37 +2014102813 product_9 pty_country_22 +2014102813 product_1 pty_country_39 +2014102813 product_5 pty_country_43 +2014102813 product_12 pty_country_15 +2014102814 product_10 pty_country_28 +2014102814 product_3 pty_country_19 +2014102814 product_9 pty_country_34 +2014102814 product_3 pty_country_36 +2014102814 product_12 pty_country_14 +2014102814 product_2 pty_country_33 +2014102814 product_6 pty_country_6 +2014102814 product_6 pty_country_12 +2014102814 product_5 pty_country_15 +2014102814 product_2 pty_country_25 +2014102815 product_7 pty_country_8 +2014102815 product_14 pty_country_45 +2014102815 product_14 pty_country_30 +2014102815 product_13 pty_country_39 +2014102815 product_7 pty_country_6 +2014102815 product_4 pty_country_34 +2014102815 product_7 pty_country_35 +2014102815 product_5 pty_country_36 +2014102815 product_6 pty_country_36 +2014102815 product_5 pty_country_40 +2014102816 product_7 pty_country_26 +2014102816 product_12 pty_country_45 +2014102816 product_8 pty_country_26 +2014102816 product_6 pty_country_19 +2014102816 product_4 pty_country_7 +2014102816 product_9 pty_country_9 +2014102816 product_14 pty_country_9 +2014102816 product_8 pty_country_31 +2014102816 product_15 pty_country_27 +2014102816 product_7 pty_country_36 +2014102817 product_10 pty_country_46 +2014102817 product_1 pty_country_39 +2014102817 product_13 pty_country_31 +2014102817 product_6 pty_country_38 +2014102817 product_10 pty_country_25 +2014102817 product_14 pty_country_37 +2014102817 product_1 pty_country_20 +2014102817 product_2 pty_country_1 +2014102817 product_6 pty_country_17 +2014102817 product_6 pty_country_43 +2014102818 product_7 pty_country_44 +2014102818 product_9 pty_country_33 +2014102818 product_7 pty_country_28 +2014102818 product_7 pty_country_32 +2014102818 product_4 pty_country_37 +2014102818 product_7 pty_country_18 +2014102818 product_1 pty_country_8 +2014102818 product_10 pty_country_45 +2014102818 product_13 pty_country_3 +2014102818 product_9 pty_country_9 +2014102819 product_1 pty_country_34 +2014102819 product_11 pty_country_37 +2014102819 product_13 pty_country_29 +2014102819 product_4 pty_country_36 +2014102819 product_13 pty_country_46 +2014102819 product_6 pty_country_16 +2014102819 product_6 pty_country_19 +2014102819 product_14 pty_country_12 +2014102819 product_6 pty_country_4 +2014102819 product_1 pty_country_36 +2014102820 product_12 pty_country_19 +2014102820 product_5 pty_country_26 +2014102820 product_14 pty_country_16 +2014102820 product_9 pty_country_5 +2014102820 product_13 pty_country_36 +2014102820 product_5 pty_country_5 +2014102820 product_5 pty_country_3 +2014102820 product_2 pty_country_42 +2014102820 product_5 pty_country_50 +2014102820 product_6 pty_country_44 +2014102821 product_8 pty_country_24 +2014102821 product_9 pty_country_14 +2014102821 product_10 pty_country_8 +2014102821 product_14 pty_country_49 +2014102821 product_7 pty_country_26 +2014102821 product_15 pty_country_31 +2014102821 product_15 pty_country_28 +2014102821 product_10 pty_country_18 +2014102821 product_14 pty_country_14 +2014102821 product_2 pty_country_30 +2014102822 product_4 pty_country_43 +2014102822 product_4 pty_country_7 +2014102822 product_11 pty_country_22 +2014102822 product_11 pty_country_8 +2014102822 product_8 pty_country_7 +2014102822 product_12 pty_country_38 +2014102822 product_10 pty_country_36 +2014102822 product_13 pty_country_49 +2014102822 product_6 pty_country_5 +2014102822 product_6 pty_country_48 +2014102823 product_12 pty_country_11 +2014102823 product_14 pty_country_20 +2014102823 product_11 pty_country_31 +2014102823 product_7 pty_country_8 +2014102823 product_12 pty_country_45 +2014102823 product_1 pty_country_41 +2014102823 product_7 pty_country_32 +2014102823 product_15 pty_country_6 +2014102823 product_5 pty_country_47 +2014102823 product_4 pty_country_18 +2014102900 product_11 pty_country_32 +2014102900 product_2 pty_country_26 +2014102900 product_2 pty_country_30 +2014102900 product_7 pty_country_22 +2014102900 product_11 pty_country_32 +2014102900 product_8 pty_country_44 +2014102900 product_3 pty_country_44 +2014102900 product_1 pty_country_37 +2014102900 product_8 pty_country_34 +2014102900 product_14 pty_country_3 +2014102901 product_12 pty_country_26 +2014102901 product_15 pty_country_29 +2014102901 product_11 pty_country_45 +2014102901 product_7 pty_country_25 +2014102901 product_11 pty_country_20 +2014102901 product_13 pty_country_38 +2014102901 product_12 pty_country_17 +2014102901 product_2 pty_country_19 +2014102901 product_6 pty_country_3 +2014102901 product_7 pty_country_39 +2014102902 product_11 pty_country_16 +2014102902 product_1 pty_country_13 +2014102902 product_7 pty_country_47 +2014102902 product_10 pty_country_40 +2014102902 product_4 pty_country_46 +2014102902 product_7 pty_country_30 +2014102902 product_13 pty_country_13 +2014102902 product_14 pty_country_48 +2014102902 product_4 pty_country_47 +2014102902 product_9 pty_country_23 +2014102903 product_14 pty_country_46 +2014102903 product_11 pty_country_2 +2014102903 product_8 pty_country_22 +2014102903 product_4 pty_country_9 +2014102903 product_12 pty_country_25 +2014102903 product_15 pty_country_43 +2014102903 product_5 pty_country_33 +2014102903 product_12 pty_country_39 +2014102903 product_9 pty_country_13 +2014102903 product_1 pty_country_24 +2014102904 product_3 pty_country_49 +2014102904 product_6 pty_country_17 +2014102904 product_6 pty_country_26 +2014102904 product_7 pty_country_9 +2014102904 product_5 pty_country_45 +2014102904 product_1 pty_country_30 +2014102904 product_1 pty_country_8 +2014102904 product_3 pty_country_19 +2014102904 product_6 pty_country_36 +2014102904 product_12 pty_country_17 +2014102905 product_5 pty_country_45 +2014102905 product_8 pty_country_14 +2014102905 product_13 pty_country_35 +2014102905 product_8 pty_country_8 +2014102905 product_5 pty_country_42 +2014102905 product_7 pty_country_7 +2014102905 product_3 pty_country_15 +2014102905 product_3 pty_country_44 +2014102905 product_9 pty_country_3 +2014102905 product_8 pty_country_49 +2014102906 product_15 pty_country_25 +2014102906 product_4 pty_country_34 +2014102906 product_1 pty_country_42 +2014102906 product_14 pty_country_42 +2014102906 product_12 pty_country_18 +2014102906 product_8 pty_country_27 +2014102906 product_5 pty_country_27 +2014102906 product_12 pty_country_45 +2014102906 product_14 pty_country_45 +2014102906 product_9 pty_country_16 +2014102907 product_2 pty_country_37 +2014102907 product_12 pty_country_48 +2014102907 product_1 pty_country_8 +2014102907 product_5 pty_country_48 +2014102907 product_9 pty_country_49 +2014102907 product_12 pty_country_1 +2014102907 product_11 pty_country_23 +2014102907 product_2 pty_country_46 +2014102907 product_5 pty_country_11 +2014102907 product_3 pty_country_4 +2014102908 product_12 pty_country_1 +2014102908 product_5 pty_country_9 +2014102908 product_12 pty_country_1 +2014102908 product_10 pty_country_9 +2014102908 product_4 pty_country_40 +2014102908 product_13 pty_country_41 +2014102908 product_13 pty_country_15 +2014102908 product_15 pty_country_45 +2014102908 product_3 pty_country_3 +2014102908 product_4 pty_country_48 +2014102909 product_5 pty_country_37 +2014102909 product_15 pty_country_30 +2014102909 product_4 pty_country_50 +2014102909 product_3 pty_country_46 +2014102909 product_2 pty_country_35 +2014102909 product_11 pty_country_28 +2014102909 product_4 pty_country_1 +2014102909 product_2 pty_country_10 +2014102909 product_9 pty_country_2 +2014102909 product_9 pty_country_11 +2014102910 product_7 pty_country_10 +2014102910 product_3 pty_country_39 +2014102910 product_8 pty_country_32 +2014102910 product_3 pty_country_30 +2014102910 product_6 pty_country_6 +2014102910 product_6 pty_country_41 +2014102910 product_15 pty_country_48 +2014102910 product_10 pty_country_2 +2014102910 product_11 pty_country_11 +2014102910 product_5 pty_country_24 +2014102911 product_3 pty_country_9 +2014102911 product_6 pty_country_20 +2014102911 product_6 pty_country_41 +2014102911 product_8 pty_country_46 +2014102911 product_12 pty_country_49 +2014102911 product_9 pty_country_40 +2014102911 product_9 pty_country_29 +2014102911 product_6 pty_country_42 +2014102911 product_15 pty_country_37 +2014102911 product_13 pty_country_39 +2014102912 product_10 pty_country_50 +2014102912 product_9 pty_country_49 +2014102912 product_6 pty_country_2 +2014102912 product_13 pty_country_33 +2014102912 product_15 pty_country_28 +2014102912 product_2 pty_country_10 +2014102912 product_15 pty_country_7 +2014102912 product_12 pty_country_1 +2014102912 product_2 pty_country_28 +2014102912 product_15 pty_country_16 +2014102913 product_2 pty_country_30 +2014102913 product_1 pty_country_19 +2014102913 product_3 pty_country_25 +2014102913 product_11 pty_country_4 +2014102913 product_12 pty_country_34 +2014102913 product_7 pty_country_44 +2014102913 product_8 pty_country_44 +2014102913 product_9 pty_country_29 +2014102913 product_10 pty_country_3 +2014102913 product_14 pty_country_50 +2014102914 product_4 pty_country_18 +2014102914 product_2 pty_country_45 +2014102914 product_11 pty_country_15 +2014102914 product_8 pty_country_13 +2014102914 product_11 pty_country_28 +2014102914 product_8 pty_country_19 +2014102914 product_7 pty_country_29 +2014102914 product_1 pty_country_33 +2014102914 product_8 pty_country_31 +2014102914 product_1 pty_country_25 +2014102915 product_12 pty_country_20 +2014102915 product_8 pty_country_48 +2014102915 product_14 pty_country_47 +2014102915 product_15 pty_country_13 +2014102915 product_5 pty_country_28 +2014102915 product_13 pty_country_32 +2014102915 product_6 pty_country_27 +2014102915 product_10 pty_country_44 +2014102915 product_15 pty_country_45 +2014102915 product_11 pty_country_30 +2014102916 product_12 pty_country_44 +2014102916 product_2 pty_country_48 +2014102916 product_11 pty_country_17 +2014102916 product_10 pty_country_4 +2014102916 product_14 pty_country_19 +2014102916 product_10 pty_country_7 +2014102916 product_10 pty_country_42 +2014102916 product_10 pty_country_19 +2014102916 product_3 pty_country_13 +2014102916 product_5 pty_country_48 +2014102917 product_8 pty_country_47 +2014102917 product_7 pty_country_27 +2014102917 product_9 pty_country_22 +2014102917 product_11 pty_country_2 +2014102917 product_11 pty_country_36 +2014102917 product_2 pty_country_19 +2014102917 product_8 pty_country_45 +2014102917 product_15 pty_country_20 +2014102917 product_4 pty_country_9 +2014102917 product_7 pty_country_49 +2014102918 product_8 pty_country_19 +2014102918 product_8 pty_country_32 +2014102918 product_9 pty_country_2 +2014102918 product_14 pty_country_47 +2014102918 product_9 pty_country_31 +2014102918 product_15 pty_country_16 +2014102918 product_15 pty_country_17 +2014102918 product_5 pty_country_20 +2014102918 product_9 pty_country_4 +2014102918 product_9 pty_country_20 +2014102919 product_8 pty_country_1 +2014102919 product_15 pty_country_15 +2014102919 product_15 pty_country_45 +2014102919 product_3 pty_country_23 +2014102919 product_15 pty_country_1 +2014102919 product_11 pty_country_43 +2014102919 product_11 pty_country_6 +2014102919 product_8 pty_country_29 +2014102919 product_15 pty_country_24 +2014102919 product_15 pty_country_45 +2014102920 product_2 pty_country_17 +2014102920 product_8 pty_country_14 +2014102920 product_2 pty_country_40 +2014102920 product_7 pty_country_39 +2014102920 product_7 pty_country_28 +2014102920 product_12 pty_country_47 +2014102920 product_13 pty_country_12 +2014102920 product_15 pty_country_31 +2014102920 product_14 pty_country_3 +2014102920 product_2 pty_country_7 +2014102921 product_1 pty_country_43 +2014102921 product_1 pty_country_44 +2014102921 product_4 pty_country_40 +2014102921 product_10 pty_country_28 +2014102921 product_5 pty_country_28 +2014102921 product_12 pty_country_6 +2014102921 product_5 pty_country_19 +2014102921 product_8 pty_country_49 +2014102921 product_6 pty_country_41 +2014102921 product_13 pty_country_25 +2014102922 product_7 pty_country_29 +2014102922 product_14 pty_country_3 +2014102922 product_3 pty_country_29 +2014102922 product_6 pty_country_8 +2014102922 product_15 pty_country_31 +2014102922 product_15 pty_country_38 +2014102922 product_4 pty_country_46 +2014102922 product_8 pty_country_34 +2014102922 product_4 pty_country_39 +2014102922 product_14 pty_country_19 +2014102923 product_11 pty_country_24 +2014102923 product_15 pty_country_24 +2014102923 product_6 pty_country_41 +2014102923 product_6 pty_country_47 +2014102923 product_4 pty_country_44 +2014102923 product_13 pty_country_29 +2014102923 product_6 pty_country_18 +2014102923 product_11 pty_country_41 +2014102923 product_4 pty_country_45 +2014102923 product_6 pty_country_37 +2014103000 product_15 pty_country_26 +2014103000 product_13 pty_country_46 +2014103000 product_1 pty_country_28 +2014103000 product_10 pty_country_15 +2014103000 product_6 pty_country_41 +2014103000 product_10 pty_country_43 +2014103000 product_15 pty_country_36 +2014103000 product_6 pty_country_7 +2014103000 product_2 pty_country_25 +2014103000 product_12 pty_country_1 +2014103001 product_13 pty_country_6 +2014103001 product_10 pty_country_40 +2014103001 product_7 pty_country_17 +2014103001 product_11 pty_country_10 +2014103001 product_10 pty_country_8 +2014103001 product_14 pty_country_37 +2014103001 product_8 pty_country_34 +2014103001 product_13 pty_country_1 +2014103001 product_12 pty_country_2 +2014103001 product_7 pty_country_5 +2014103002 product_9 pty_country_33 +2014103002 product_2 pty_country_39 +2014103002 product_4 pty_country_19 +2014103002 product_8 pty_country_43 +2014103002 product_3 pty_country_20 +2014103002 product_12 pty_country_48 +2014103002 product_10 pty_country_35 +2014103002 product_3 pty_country_12 +2014103002 product_5 pty_country_5 +2014103002 product_5 pty_country_36 +2014103003 product_8 pty_country_27 +2014103003 product_2 pty_country_43 +2014103003 product_9 pty_country_11 +2014103003 product_10 pty_country_45 +2014103003 product_11 pty_country_12 +2014103003 product_15 pty_country_5 +2014103003 product_7 pty_country_17 +2014103003 product_6 pty_country_22 +2014103003 product_6 pty_country_18 +2014103003 product_14 pty_country_25 +2014103004 product_13 pty_country_28 +2014103004 product_12 pty_country_21 +2014103004 product_11 pty_country_31 +2014103004 product_2 pty_country_8 +2014103004 product_3 pty_country_9 +2014103004 product_8 pty_country_10 +2014103004 product_10 pty_country_21 +2014103004 product_11 pty_country_25 +2014103004 product_7 pty_country_39 +2014103004 product_12 pty_country_24 +2014103005 product_10 pty_country_33 +2014103005 product_15 pty_country_10 +2014103005 product_11 pty_country_5 +2014103005 product_1 pty_country_42 +2014103005 product_10 pty_country_26 +2014103005 product_11 pty_country_4 +2014103005 product_13 pty_country_39 +2014103005 product_8 pty_country_19 +2014103005 product_7 pty_country_23 +2014103005 product_11 pty_country_44 +2014103006 product_10 pty_country_7 +2014103006 product_2 pty_country_30 +2014103006 product_13 pty_country_32 +2014103006 product_3 pty_country_15 +2014103006 product_7 pty_country_31 +2014103006 product_13 pty_country_12 +2014103006 product_3 pty_country_20 +2014103006 product_12 pty_country_19 +2014103006 product_2 pty_country_44 +2014103006 product_5 pty_country_7 +2014103007 product_12 pty_country_13 +2014103007 product_3 pty_country_10 +2014103007 product_7 pty_country_3 +2014103007 product_6 pty_country_43 +2014103007 product_6 pty_country_27 +2014103007 product_13 pty_country_22 +2014103007 product_8 pty_country_28 +2014103007 product_3 pty_country_7 +2014103007 product_12 pty_country_42 +2014103007 product_12 pty_country_38 +2014103008 product_5 pty_country_26 +2014103008 product_15 pty_country_8 +2014103008 product_11 pty_country_42 +2014103008 product_8 pty_country_41 +2014103008 product_14 pty_country_32 +2014103008 product_11 pty_country_34 +2014103008 product_10 pty_country_6 +2014103008 product_14 pty_country_3 +2014103008 product_13 pty_country_4 +2014103008 product_14 pty_country_40 +2014103009 product_12 pty_country_17 +2014103009 product_12 pty_country_48 +2014103009 product_9 pty_country_45 +2014103009 product_8 pty_country_49 +2014103009 product_15 pty_country_4 +2014103009 product_5 pty_country_43 +2014103009 product_7 pty_country_37 +2014103009 product_1 pty_country_48 +2014103009 product_9 pty_country_39 +2014103009 product_6 pty_country_36 +2014103010 product_7 pty_country_37 +2014103010 product_9 pty_country_24 +2014103010 product_11 pty_country_32 +2014103010 product_6 pty_country_21 +2014103010 product_8 pty_country_47 +2014103010 product_3 pty_country_21 +2014103010 product_10 pty_country_48 +2014103010 product_8 pty_country_7 +2014103010 product_2 pty_country_22 +2014103010 product_6 pty_country_34 +2014103011 product_13 pty_country_21 +2014103011 product_11 pty_country_28 +2014103011 product_5 pty_country_21 +2014103011 product_13 pty_country_3 +2014103011 product_10 pty_country_18 +2014103011 product_5 pty_country_35 +2014103011 product_6 pty_country_33 +2014103011 product_12 pty_country_9 +2014103011 product_10 pty_country_22 +2014103011 product_3 pty_country_47 +2014103012 product_15 pty_country_32 +2014103012 product_14 pty_country_23 +2014103012 product_4 pty_country_19 +2014103012 product_1 pty_country_45 +2014103012 product_15 pty_country_17 +2014103012 product_13 pty_country_31 +2014103012 product_5 pty_country_46 +2014103012 product_2 pty_country_40 +2014103012 product_2 pty_country_31 +2014103012 product_7 pty_country_22 +2014103013 product_1 pty_country_45 +2014103013 product_8 pty_country_33 +2014103013 product_1 pty_country_40 +2014103013 product_10 pty_country_12 +2014103013 product_12 pty_country_23 +2014103013 product_1 pty_country_13 +2014103013 product_14 pty_country_28 +2014103013 product_2 pty_country_5 +2014103013 product_8 pty_country_40 +2014103013 product_5 pty_country_18 +2014103014 product_1 pty_country_22 +2014103014 product_14 pty_country_40 +2014103014 product_10 pty_country_36 +2014103014 product_8 pty_country_14 +2014103014 product_1 pty_country_36 +2014103014 product_3 pty_country_46 +2014103014 product_6 pty_country_29 +2014103014 product_13 pty_country_28 +2014103014 product_2 pty_country_15 +2014103014 product_14 pty_country_20 +2014103015 product_13 pty_country_32 +2014103015 product_2 pty_country_5 +2014103015 product_10 pty_country_13 +2014103015 product_10 pty_country_6 +2014103015 product_8 pty_country_10 +2014103015 product_3 pty_country_27 +2014103015 product_1 pty_country_31 +2014103015 product_9 pty_country_3 +2014103015 product_10 pty_country_25 +2014103015 product_11 pty_country_44 +2014103016 product_12 pty_country_33 +2014103016 product_13 pty_country_15 +2014103016 product_1 pty_country_33 +2014103016 product_5 pty_country_19 +2014103016 product_1 pty_country_23 +2014103016 product_15 pty_country_29 +2014103016 product_12 pty_country_4 +2014103016 product_4 pty_country_34 +2014103016 product_5 pty_country_27 +2014103016 product_11 pty_country_36 +2014103017 product_12 pty_country_1 +2014103017 product_2 pty_country_1 +2014103017 product_2 pty_country_10 +2014103017 product_11 pty_country_1 +2014103017 product_2 pty_country_27 +2014103017 product_14 pty_country_6 +2014103017 product_11 pty_country_6 +2014103017 product_1 pty_country_10 +2014103017 product_2 pty_country_44 +2014103017 product_6 pty_country_47 +2014103018 product_14 pty_country_31 +2014103018 product_3 pty_country_42 +2014103018 product_1 pty_country_44 +2014103018 product_11 pty_country_4 +2014103018 product_13 pty_country_46 +2014103018 product_3 pty_country_27 +2014103018 product_13 pty_country_28 +2014103018 product_8 pty_country_8 +2014103018 product_7 pty_country_20 +2014103018 product_12 pty_country_42 +2014103019 product_1 pty_country_31 +2014103019 product_5 pty_country_14 +2014103019 product_4 pty_country_40 +2014103019 product_15 pty_country_7 +2014103019 product_15 pty_country_9 +2014103019 product_9 pty_country_8 +2014103019 product_12 pty_country_4 +2014103019 product_1 pty_country_7 +2014103019 product_2 pty_country_45 +2014103019 product_8 pty_country_38 +2014103020 product_5 pty_country_22 +2014103020 product_8 pty_country_34 +2014103020 product_11 pty_country_16 +2014103020 product_6 pty_country_34 +2014103020 product_10 pty_country_11 +2014103020 product_14 pty_country_35 +2014103020 product_13 pty_country_22 +2014103020 product_1 pty_country_27 +2014103020 product_12 pty_country_43 +2014103020 product_13 pty_country_30 +2014103021 product_7 pty_country_50 +2014103021 product_11 pty_country_20 +2014103021 product_5 pty_country_33 +2014103021 product_10 pty_country_26 +2014103021 product_8 pty_country_26 +2014103021 product_13 pty_country_44 +2014103021 product_12 pty_country_40 +2014103021 product_12 pty_country_38 +2014103021 product_3 pty_country_4 +2014103021 product_5 pty_country_27 +2014103022 product_4 pty_country_26 +2014103022 product_10 pty_country_4 +2014103022 product_11 pty_country_32 +2014103022 product_12 pty_country_8 +2014103022 product_8 pty_country_49 +2014103022 product_13 pty_country_14 +2014103022 product_5 pty_country_10 +2014103022 product_7 pty_country_24 +2014103022 product_15 pty_country_20 +2014103022 product_15 pty_country_22 +2014103023 product_1 pty_country_30 +2014103023 product_13 pty_country_29 +2014103023 product_15 pty_country_26 +2014103023 product_13 pty_country_39 +2014103023 product_12 pty_country_31 +2014103023 product_12 pty_country_22 +2014103023 product_14 pty_country_3 +2014103023 product_8 pty_country_49 +2014103023 product_1 pty_country_38 +2014103023 product_8 pty_country_38 +2014103100 product_1 pty_country_22 +2014103100 product_8 pty_country_28 +2014103100 product_7 pty_country_33 +2014103100 product_12 pty_country_36 +2014103100 product_8 pty_country_49 +2014103100 product_10 pty_country_33 +2014103100 product_7 pty_country_49 +2014103100 product_8 pty_country_44 +2014103100 product_12 pty_country_44 +2014103100 product_1 pty_country_50 +2014103101 product_15 pty_country_24 +2014103101 product_4 pty_country_10 +2014103101 product_1 pty_country_21 +2014103101 product_15 pty_country_5 +2014103101 product_14 pty_country_49 +2014103101 product_5 pty_country_18 +2014103101 product_1 pty_country_40 +2014103101 product_14 pty_country_2 +2014103101 product_3 pty_country_25 +2014103101 product_10 pty_country_44 +2014103102 product_6 pty_country_47 +2014103102 product_9 pty_country_27 +2014103102 product_12 pty_country_38 +2014103102 product_12 pty_country_24 +2014103102 product_3 pty_country_33 +2014103102 product_8 pty_country_50 +2014103102 product_14 pty_country_44 +2014103102 product_3 pty_country_27 +2014103102 product_5 pty_country_47 +2014103102 product_14 pty_country_45 +2014103103 product_6 pty_country_24 +2014103103 product_4 pty_country_17 +2014103103 product_14 pty_country_32 +2014103103 product_7 pty_country_21 +2014103103 product_11 pty_country_9 +2014103103 product_3 pty_country_18 +2014103103 product_14 pty_country_11 +2014103103 product_10 pty_country_42 +2014103103 product_7 pty_country_32 +2014103103 product_6 pty_country_35 +2014103104 product_8 pty_country_31 +2014103104 product_5 pty_country_28 +2014103104 product_12 pty_country_15 +2014103104 product_11 pty_country_44 +2014103104 product_12 pty_country_11 +2014103104 product_7 pty_country_8 +2014103104 product_6 pty_country_1 +2014103104 product_9 pty_country_36 +2014103104 product_9 pty_country_8 +2014103104 product_5 pty_country_50 +2014103105 product_14 pty_country_49 +2014103105 product_11 pty_country_1 +2014103105 product_1 pty_country_34 +2014103105 product_13 pty_country_6 +2014103105 product_1 pty_country_4 +2014103105 product_9 pty_country_45 +2014103105 product_2 pty_country_24 +2014103105 product_3 pty_country_36 +2014103105 product_12 pty_country_19 +2014103105 product_2 pty_country_29 +2014103106 product_1 pty_country_17 +2014103106 product_1 pty_country_19 +2014103106 product_3 pty_country_24 +2014103106 product_2 pty_country_34 +2014103106 product_14 pty_country_1 +2014103106 product_2 pty_country_7 +2014103106 product_14 pty_country_27 +2014103106 product_2 pty_country_49 +2014103106 product_1 pty_country_13 +2014103106 product_6 pty_country_19 +2014103107 product_8 pty_country_41 +2014103107 product_12 pty_country_48 +2014103107 product_2 pty_country_11 +2014103107 product_1 pty_country_5 +2014103107 product_12 pty_country_27 +2014103107 product_14 pty_country_16 +2014103107 product_3 pty_country_50 +2014103107 product_8 pty_country_29 +2014103107 product_1 pty_country_13 +2014103107 product_13 pty_country_35 +2014103108 product_13 pty_country_18 +2014103108 product_15 pty_country_10 +2014103108 product_6 pty_country_4 +2014103108 product_10 pty_country_8 +2014103108 product_15 pty_country_29 +2014103108 product_4 pty_country_2 +2014103108 product_3 pty_country_13 +2014103108 product_14 pty_country_7 +2014103108 product_6 pty_country_11 +2014103108 product_6 pty_country_11 +2014103109 product_11 pty_country_13 +2014103109 product_15 pty_country_44 +2014103109 product_8 pty_country_16 +2014103109 product_1 pty_country_1 +2014103109 product_11 pty_country_8 +2014103109 product_13 pty_country_29 +2014103109 product_15 pty_country_7 +2014103109 product_8 pty_country_12 +2014103109 product_11 pty_country_48 +2014103109 product_11 pty_country_13 +2014103110 product_9 pty_country_41 +2014103110 product_6 pty_country_11 +2014103110 product_8 pty_country_42 +2014103110 product_6 pty_country_1 +2014103110 product_14 pty_country_39 +2014103110 product_12 pty_country_36 +2014103110 product_14 pty_country_35 +2014103110 product_11 pty_country_11 +2014103110 product_8 pty_country_17 +2014103110 product_2 pty_country_15 +2014103111 product_8 pty_country_28 +2014103111 product_6 pty_country_50 +2014103111 product_7 pty_country_13 +2014103111 product_5 pty_country_22 +2014103111 product_2 pty_country_23 +2014103111 product_3 pty_country_40 +2014103111 product_9 pty_country_2 +2014103111 product_7 pty_country_26 +2014103111 product_4 pty_country_8 +2014103111 product_2 pty_country_21 +2014103112 product_1 pty_country_43 +2014103112 product_6 pty_country_12 +2014103112 product_9 pty_country_18 +2014103112 product_11 pty_country_13 +2014103112 product_8 pty_country_22 +2014103112 product_13 pty_country_17 +2014103112 product_9 pty_country_15 +2014103112 product_5 pty_country_26 +2014103112 product_7 pty_country_7 +2014103112 product_15 pty_country_45 +2014103113 product_5 pty_country_19 +2014103113 product_13 pty_country_8 +2014103113 product_14 pty_country_31 +2014103113 product_4 pty_country_4 +2014103113 product_9 pty_country_42 +2014103113 product_12 pty_country_4 +2014103113 product_1 pty_country_29 +2014103113 product_13 pty_country_38 +2014103113 product_15 pty_country_1 +2014103113 product_12 pty_country_28 +2014103114 product_9 pty_country_44 +2014103114 product_7 pty_country_25 +2014103114 product_2 pty_country_30 +2014103114 product_3 pty_country_13 +2014103114 product_4 pty_country_46 +2014103114 product_3 pty_country_9 +2014103114 product_2 pty_country_44 +2014103114 product_14 pty_country_44 +2014103114 product_10 pty_country_45 +2014103114 product_4 pty_country_24 +2014103115 product_12 pty_country_35 +2014103115 product_11 pty_country_8 +2014103115 product_15 pty_country_2 +2014103115 product_8 pty_country_37 +2014103115 product_10 pty_country_45 +2014103115 product_10 pty_country_39 +2014103115 product_10 pty_country_20 +2014103115 product_13 pty_country_39 +2014103115 product_12 pty_country_33 +2014103115 product_1 pty_country_13 +2014103116 product_4 pty_country_40 +2014103116 product_10 pty_country_3 +2014103116 product_14 pty_country_48 +2014103116 product_11 pty_country_41 +2014103116 product_4 pty_country_40 +2014103116 product_2 pty_country_36 +2014103116 product_15 pty_country_5 +2014103116 product_7 pty_country_2 +2014103116 product_7 pty_country_1 +2014103116 product_5 pty_country_34 +2014103117 product_9 pty_country_14 +2014103117 product_5 pty_country_37 +2014103117 product_14 pty_country_45 +2014103117 product_2 pty_country_5 +2014103117 product_4 pty_country_6 +2014103117 product_13 pty_country_11 +2014103117 product_8 pty_country_38 +2014103117 product_12 pty_country_43 +2014103117 product_7 pty_country_14 +2014103117 product_8 pty_country_29 +2014103118 product_6 pty_country_21 +2014103118 product_9 pty_country_48 +2014103118 product_3 pty_country_35 +2014103118 product_3 pty_country_42 +2014103118 product_1 pty_country_9 +2014103118 product_12 pty_country_36 +2014103118 product_11 pty_country_39 +2014103118 product_6 pty_country_50 +2014103118 product_1 pty_country_15 +2014103118 product_6 pty_country_43 +2014103119 product_12 pty_country_48 +2014103119 product_10 pty_country_43 +2014103119 product_11 pty_country_19 +2014103119 product_10 pty_country_11 +2014103119 product_10 pty_country_39 +2014103119 product_11 pty_country_42 +2014103119 product_3 pty_country_24 +2014103119 product_15 pty_country_15 +2014103119 product_7 pty_country_24 +2014103119 product_1 pty_country_17 +2014103120 product_6 pty_country_36 +2014103120 product_13 pty_country_5 +2014103120 product_15 pty_country_32 +2014103120 product_4 pty_country_5 +2014103120 product_11 pty_country_40 +2014103120 product_14 pty_country_42 +2014103120 product_7 pty_country_36 +2014103120 product_14 pty_country_5 +2014103120 product_1 pty_country_25 +2014103120 product_6 pty_country_8 +2014103121 product_3 pty_country_9 +2014103121 product_11 pty_country_43 +2014103121 product_8 pty_country_14 +2014103121 product_12 pty_country_9 +2014103121 product_9 pty_country_37 +2014103121 product_5 pty_country_9 +2014103121 product_14 pty_country_40 +2014103121 product_14 pty_country_50 +2014103121 product_8 pty_country_15 +2014103121 product_13 pty_country_20 +2014103122 product_4 pty_country_44 +2014103122 product_13 pty_country_10 +2014103122 product_9 pty_country_44 +2014103122 product_1 pty_country_46 +2014103122 product_5 pty_country_36 +2014103122 product_11 pty_country_20 +2014103122 product_13 pty_country_8 +2014103122 product_11 pty_country_45 +2014103122 product_10 pty_country_26 +2014103122 product_9 pty_country_14 +2014103123 product_7 pty_country_41 +2014103123 product_8 pty_country_6 +2014103123 product_12 pty_country_13 +2014103123 product_5 pty_country_23 +2014103123 product_1 pty_country_20 +2014103123 product_14 pty_country_20 +2014103123 product_1 pty_country_5 +2014103123 product_6 pty_country_35 +2014103123 product_12 pty_country_29 +2014103123 product_4 pty_country_35 +2014110100 product_14 pty_country_37 +2014110100 product_13 pty_country_13 +2014110100 product_12 pty_country_38 +2014110100 product_8 pty_country_13 +2014110100 product_14 pty_country_23 +2014110100 product_1 pty_country_45 +2014110100 product_6 pty_country_26 +2014110100 product_9 pty_country_18 +2014110100 product_6 pty_country_4 +2014110100 product_13 pty_country_28 +2014110101 product_11 pty_country_38 +2014110101 product_11 pty_country_26 +2014110101 product_11 pty_country_9 +2014110101 product_9 pty_country_29 +2014110101 product_4 pty_country_25 +2014110101 product_5 pty_country_49 +2014110101 product_5 pty_country_40 +2014110101 product_7 pty_country_11 +2014110101 product_7 pty_country_2 +2014110101 product_13 pty_country_33 +2014110102 product_2 pty_country_11 +2014110102 product_12 pty_country_4 +2014110102 product_13 pty_country_16 +2014110102 product_12 pty_country_42 +2014110102 product_2 pty_country_12 +2014110102 product_5 pty_country_9 +2014110102 product_7 pty_country_14 +2014110102 product_6 pty_country_26 +2014110102 product_14 pty_country_47 +2014110102 product_5 pty_country_6 +2014110103 product_15 pty_country_11 +2014110103 product_4 pty_country_26 +2014110103 product_2 pty_country_31 +2014110103 product_7 pty_country_29 +2014110103 product_7 pty_country_41 +2014110103 product_15 pty_country_44 +2014110103 product_1 pty_country_3 +2014110103 product_15 pty_country_6 +2014110103 product_1 pty_country_13 +2014110103 product_15 pty_country_46 +2014110104 product_3 pty_country_28 +2014110104 product_3 pty_country_24 +2014110104 product_2 pty_country_11 +2014110104 product_10 pty_country_43 +2014110104 product_11 pty_country_40 +2014110104 product_12 pty_country_25 +2014110104 product_14 pty_country_3 +2014110104 product_10 pty_country_25 +2014110104 product_2 pty_country_34 +2014110104 product_6 pty_country_37 +2014110105 product_9 pty_country_40 +2014110105 product_7 pty_country_37 +2014110105 product_2 pty_country_40 +2014110105 product_4 pty_country_29 +2014110105 product_7 pty_country_21 +2014110105 product_7 pty_country_33 +2014110105 product_1 pty_country_34 +2014110105 product_7 pty_country_17 +2014110105 product_11 pty_country_44 +2014110105 product_10 pty_country_34 +2014110106 product_1 pty_country_45 +2014110106 product_1 pty_country_48 +2014110106 product_5 pty_country_15 +2014110106 product_12 pty_country_17 +2014110106 product_9 pty_country_12 +2014110106 product_9 pty_country_17 +2014110106 product_1 pty_country_29 +2014110106 product_2 pty_country_45 +2014110106 product_1 pty_country_16 +2014110106 product_15 pty_country_48 +2014110107 product_11 pty_country_11 +2014110107 product_1 pty_country_43 +2014110107 product_12 pty_country_46 +2014110107 product_8 pty_country_23 +2014110107 product_6 pty_country_18 +2014110107 product_14 pty_country_39 +2014110107 product_7 pty_country_37 +2014110107 product_1 pty_country_31 +2014110107 product_3 pty_country_43 +2014110107 product_1 pty_country_17 +2014110108 product_5 pty_country_4 +2014110108 product_8 pty_country_27 +2014110108 product_5 pty_country_37 +2014110108 product_2 pty_country_48 +2014110108 product_8 pty_country_17 +2014110108 product_9 pty_country_9 +2014110108 product_1 pty_country_30 +2014110108 product_8 pty_country_4 +2014110108 product_13 pty_country_30 +2014110108 product_13 pty_country_47 +2014110109 product_1 pty_country_44 +2014110109 product_13 pty_country_39 +2014110109 product_6 pty_country_24 +2014110109 product_8 pty_country_40 +2014110109 product_4 pty_country_22 +2014110109 product_6 pty_country_31 +2014110109 product_12 pty_country_23 +2014110109 product_7 pty_country_38 +2014110109 product_12 pty_country_23 +2014110109 product_4 pty_country_24 +2014110110 product_2 pty_country_32 +2014110110 product_14 pty_country_17 +2014110110 product_5 pty_country_34 +2014110110 product_14 pty_country_12 +2014110110 product_6 pty_country_1 +2014110110 product_2 pty_country_18 +2014110110 product_3 pty_country_50 +2014110110 product_13 pty_country_34 +2014110110 product_12 pty_country_46 +2014110110 product_5 pty_country_40 +2014110111 product_7 pty_country_16 +2014110111 product_12 pty_country_47 +2014110111 product_11 pty_country_26 +2014110111 product_1 pty_country_11 +2014110111 product_14 pty_country_3 +2014110111 product_3 pty_country_6 +2014110111 product_9 pty_country_26 +2014110111 product_15 pty_country_6 +2014110111 product_9 pty_country_20 +2014110111 product_14 pty_country_5 +2014110112 product_9 pty_country_10 +2014110112 product_10 pty_country_11 +2014110112 product_6 pty_country_20 +2014110112 product_4 pty_country_21 +2014110112 product_4 pty_country_11 +2014110112 product_7 pty_country_37 +2014110112 product_14 pty_country_31 +2014110112 product_13 pty_country_27 +2014110112 product_13 pty_country_37 +2014110112 product_4 pty_country_1 +2014110113 product_11 pty_country_5 +2014110113 product_6 pty_country_27 +2014110113 product_11 pty_country_41 +2014110113 product_15 pty_country_7 +2014110113 product_7 pty_country_27 +2014110113 product_13 pty_country_24 +2014110113 product_12 pty_country_15 +2014110113 product_10 pty_country_9 +2014110113 product_7 pty_country_6 +2014110113 product_10 pty_country_44 +2014110114 product_3 pty_country_24 +2014110114 product_15 pty_country_18 +2014110114 product_2 pty_country_9 +2014110114 product_6 pty_country_8 +2014110114 product_1 pty_country_38 +2014110114 product_13 pty_country_25 +2014110114 product_10 pty_country_31 +2014110114 product_3 pty_country_16 +2014110114 product_2 pty_country_7 +2014110114 product_11 pty_country_44 +2014110115 product_8 pty_country_22 +2014110115 product_13 pty_country_10 +2014110115 product_4 pty_country_36 +2014110115 product_13 pty_country_9 +2014110115 product_2 pty_country_27 +2014110115 product_7 pty_country_44 +2014110115 product_11 pty_country_28 +2014110115 product_3 pty_country_17 +2014110115 product_3 pty_country_42 +2014110115 product_2 pty_country_46 +2014110116 product_2 pty_country_13 +2014110116 product_11 pty_country_17 +2014110116 product_14 pty_country_35 +2014110116 product_1 pty_country_38 +2014110116 product_5 pty_country_9 +2014110116 product_9 pty_country_21 +2014110116 product_6 pty_country_10 +2014110116 product_11 pty_country_23 +2014110116 product_11 pty_country_41 +2014110116 product_2 pty_country_3 +2014110117 product_2 pty_country_46 +2014110117 product_6 pty_country_24 +2014110117 product_8 pty_country_31 +2014110117 product_7 pty_country_26 +2014110117 product_9 pty_country_44 +2014110117 product_9 pty_country_21 +2014110117 product_1 pty_country_41 +2014110117 product_14 pty_country_17 +2014110117 product_2 pty_country_31 +2014110117 product_7 pty_country_35 +2014110118 product_15 pty_country_34 +2014110118 product_7 pty_country_31 +2014110118 product_5 pty_country_2 +2014110118 product_4 pty_country_15 +2014110118 product_9 pty_country_26 +2014110118 product_7 pty_country_2 +2014110118 product_1 pty_country_16 +2014110118 product_2 pty_country_45 +2014110118 product_6 pty_country_7 +2014110118 product_6 pty_country_27 +2014110119 product_13 pty_country_15 +2014110119 product_14 pty_country_2 +2014110119 product_13 pty_country_21 +2014110119 product_8 pty_country_16 +2014110119 product_2 pty_country_41 +2014110119 product_7 pty_country_34 +2014110119 product_5 pty_country_21 +2014110119 product_13 pty_country_13 +2014110119 product_13 pty_country_12 +2014110119 product_14 pty_country_49 +2014110120 product_9 pty_country_36 +2014110120 product_2 pty_country_30 +2014110120 product_1 pty_country_8 +2014110120 product_6 pty_country_33 +2014110120 product_15 pty_country_23 +2014110120 product_15 pty_country_1 +2014110120 product_9 pty_country_19 +2014110120 product_13 pty_country_36 +2014110120 product_6 pty_country_21 +2014110120 product_10 pty_country_41 +2014110121 product_12 pty_country_28 +2014110121 product_13 pty_country_11 +2014110121 product_5 pty_country_50 +2014110121 product_5 pty_country_30 +2014110121 product_6 pty_country_41 +2014110121 product_12 pty_country_8 +2014110121 product_4 pty_country_9 +2014110121 product_2 pty_country_5 +2014110121 product_3 pty_country_6 +2014110121 product_2 pty_country_40 +2014110122 product_12 pty_country_16 +2014110122 product_11 pty_country_49 +2014110122 product_4 pty_country_45 +2014110122 product_1 pty_country_2 +2014110122 product_7 pty_country_46 +2014110122 product_15 pty_country_39 +2014110122 product_9 pty_country_23 +2014110122 product_7 pty_country_16 +2014110122 product_14 pty_country_23 +2014110122 product_3 pty_country_37 +2014110123 product_7 pty_country_47 +2014110123 product_6 pty_country_40 +2014110123 product_10 pty_country_32 +2014110123 product_7 pty_country_21 +2014110123 product_8 pty_country_19 +2014110123 product_12 pty_country_47 +2014110123 product_5 pty_country_28 +2014110123 product_7 pty_country_11 +2014110123 product_9 pty_country_24 +2014110123 product_5 pty_country_44 +2014110200 product_10 pty_country_36 +2014110200 product_7 pty_country_13 +2014110200 product_7 pty_country_21 +2014110200 product_7 pty_country_6 +2014110200 product_4 pty_country_13 +2014110200 product_12 pty_country_15 +2014110200 product_5 pty_country_23 +2014110200 product_7 pty_country_29 +2014110200 product_6 pty_country_27 +2014110200 product_13 pty_country_19 +2014110201 product_7 pty_country_1 +2014110201 product_14 pty_country_47 +2014110201 product_1 pty_country_24 +2014110201 product_14 pty_country_43 +2014110201 product_5 pty_country_10 +2014110201 product_12 pty_country_2 +2014110201 product_4 pty_country_43 +2014110201 product_4 pty_country_47 +2014110201 product_3 pty_country_21 +2014110201 product_5 pty_country_33 +2014110202 product_14 pty_country_41 +2014110202 product_11 pty_country_37 +2014110202 product_2 pty_country_20 +2014110202 product_4 pty_country_12 +2014110202 product_12 pty_country_38 +2014110202 product_14 pty_country_14 +2014110202 product_10 pty_country_12 +2014110202 product_8 pty_country_31 +2014110202 product_10 pty_country_14 +2014110202 product_1 pty_country_26 +2014110203 product_4 pty_country_4 +2014110203 product_15 pty_country_30 +2014110203 product_6 pty_country_49 +2014110203 product_11 pty_country_13 +2014110203 product_12 pty_country_3 +2014110203 product_7 pty_country_20 +2014110203 product_8 pty_country_46 +2014110203 product_7 pty_country_18 +2014110203 product_14 pty_country_6 +2014110203 product_2 pty_country_9 +2014110204 product_13 pty_country_11 +2014110204 product_7 pty_country_28 +2014110204 product_13 pty_country_16 +2014110204 product_6 pty_country_36 +2014110204 product_6 pty_country_8 +2014110204 product_7 pty_country_7 +2014110204 product_12 pty_country_18 +2014110204 product_3 pty_country_35 +2014110204 product_2 pty_country_26 +2014110204 product_6 pty_country_43 +2014110205 product_6 pty_country_33 +2014110205 product_8 pty_country_20 +2014110205 product_5 pty_country_43 +2014110205 product_14 pty_country_10 +2014110205 product_10 pty_country_9 +2014110205 product_8 pty_country_17 +2014110205 product_4 pty_country_49 +2014110205 product_6 pty_country_27 +2014110205 product_4 pty_country_35 +2014110205 product_8 pty_country_37 +2014110206 product_8 pty_country_47 +2014110206 product_12 pty_country_18 +2014110206 product_12 pty_country_46 +2014110206 product_12 pty_country_47 +2014110206 product_3 pty_country_10 +2014110206 product_3 pty_country_40 +2014110206 product_5 pty_country_8 +2014110206 product_1 pty_country_27 +2014110206 product_5 pty_country_31 +2014110206 product_4 pty_country_14 +2014110207 product_10 pty_country_31 +2014110207 product_7 pty_country_15 +2014110207 product_12 pty_country_23 +2014110207 product_9 pty_country_37 +2014110207 product_8 pty_country_10 +2014110207 product_8 pty_country_46 +2014110207 product_4 pty_country_39 +2014110207 product_4 pty_country_31 +2014110207 product_13 pty_country_10 +2014110207 product_13 pty_country_38 +2014110208 product_9 pty_country_3 +2014110208 product_15 pty_country_10 +2014110208 product_1 pty_country_36 +2014110208 product_1 pty_country_3 +2014110208 product_9 pty_country_46 +2014110208 product_13 pty_country_20 +2014110208 product_10 pty_country_38 +2014110208 product_14 pty_country_48 +2014110208 product_12 pty_country_37 +2014110208 product_9 pty_country_49 +2014110209 product_2 pty_country_32 +2014110209 product_13 pty_country_39 +2014110209 product_2 pty_country_4 +2014110209 product_9 pty_country_31 +2014110209 product_5 pty_country_10 +2014110209 product_10 pty_country_11 +2014110209 product_3 pty_country_15 +2014110209 product_11 pty_country_44 +2014110209 product_1 pty_country_16 +2014110209 product_10 pty_country_48 +2014110210 product_10 pty_country_41 +2014110210 product_3 pty_country_27 +2014110210 product_14 pty_country_1 +2014110210 product_3 pty_country_20 +2014110210 product_12 pty_country_24 +2014110210 product_10 pty_country_13 +2014110210 product_9 pty_country_27 +2014110210 product_11 pty_country_20 +2014110210 product_11 pty_country_23 +2014110210 product_7 pty_country_15 +2014110211 product_3 pty_country_17 +2014110211 product_7 pty_country_9 +2014110211 product_10 pty_country_13 +2014110211 product_14 pty_country_40 +2014110211 product_7 pty_country_38 +2014110211 product_9 pty_country_20 +2014110211 product_9 pty_country_32 +2014110211 product_2 pty_country_47 +2014110211 product_4 pty_country_31 +2014110211 product_7 pty_country_35 +2014110212 product_4 pty_country_45 +2014110212 product_2 pty_country_28 +2014110212 product_13 pty_country_16 +2014110212 product_2 pty_country_40 +2014110212 product_7 pty_country_44 +2014110212 product_5 pty_country_2 +2014110212 product_10 pty_country_22 +2014110212 product_3 pty_country_15 +2014110212 product_10 pty_country_2 +2014110212 product_5 pty_country_30 +2014110213 product_2 pty_country_23 +2014110213 product_2 pty_country_47 +2014110213 product_8 pty_country_12 +2014110213 product_6 pty_country_29 +2014110213 product_13 pty_country_4 +2014110213 product_10 pty_country_9 +2014110213 product_10 pty_country_37 +2014110213 product_4 pty_country_44 +2014110213 product_13 pty_country_21 +2014110213 product_14 pty_country_4 +2014110214 product_15 pty_country_45 +2014110214 product_15 pty_country_29 +2014110214 product_8 pty_country_48 +2014110214 product_2 pty_country_14 +2014110214 product_9 pty_country_10 +2014110214 product_8 pty_country_24 +2014110214 product_8 pty_country_19 +2014110214 product_4 pty_country_6 +2014110214 product_7 pty_country_33 +2014110214 product_10 pty_country_16 +2014110215 product_11 pty_country_37 +2014110215 product_1 pty_country_18 +2014110215 product_3 pty_country_11 +2014110215 product_11 pty_country_12 +2014110215 product_1 pty_country_35 +2014110215 product_11 pty_country_1 +2014110215 product_4 pty_country_16 +2014110215 product_5 pty_country_30 +2014110215 product_6 pty_country_21 +2014110215 product_5 pty_country_4 +2014110216 product_4 pty_country_43 +2014110216 product_15 pty_country_11 +2014110216 product_1 pty_country_19 +2014110216 product_5 pty_country_40 +2014110216 product_4 pty_country_38 +2014110216 product_14 pty_country_48 +2014110216 product_9 pty_country_41 +2014110216 product_12 pty_country_31 +2014110216 product_7 pty_country_37 +2014110216 product_4 pty_country_21 +2014110217 product_12 pty_country_29 +2014110217 product_15 pty_country_15 +2014110217 product_14 pty_country_1 +2014110217 product_9 pty_country_48 +2014110217 product_10 pty_country_26 +2014110217 product_9 pty_country_8 +2014110217 product_6 pty_country_2 +2014110217 product_2 pty_country_44 +2014110217 product_5 pty_country_38 +2014110217 product_2 pty_country_17 +2014110218 product_13 pty_country_23 +2014110218 product_8 pty_country_4 +2014110218 product_8 pty_country_37 +2014110218 product_15 pty_country_42 +2014110218 product_5 pty_country_28 +2014110218 product_2 pty_country_12 +2014110218 product_12 pty_country_20 +2014110218 product_12 pty_country_33 +2014110218 product_13 pty_country_7 +2014110218 product_9 pty_country_25 +2014110219 product_2 pty_country_50 +2014110219 product_3 pty_country_33 +2014110219 product_3 pty_country_40 +2014110219 product_2 pty_country_34 +2014110219 product_6 pty_country_39 +2014110219 product_8 pty_country_6 +2014110219 product_8 pty_country_39 +2014110219 product_11 pty_country_27 +2014110219 product_11 pty_country_48 +2014110219 product_12 pty_country_33 +2014110220 product_8 pty_country_3 +2014110220 product_2 pty_country_27 +2014110220 product_7 pty_country_19 +2014110220 product_1 pty_country_1 +2014110220 product_5 pty_country_3 +2014110220 product_12 pty_country_12 +2014110220 product_10 pty_country_24 +2014110220 product_9 pty_country_37 +2014110220 product_1 pty_country_2 +2014110220 product_3 pty_country_34 +2014110221 product_1 pty_country_39 +2014110221 product_14 pty_country_34 +2014110221 product_8 pty_country_19 +2014110221 product_13 pty_country_36 +2014110221 product_1 pty_country_44 +2014110221 product_13 pty_country_34 +2014110221 product_12 pty_country_17 +2014110221 product_2 pty_country_21 +2014110221 product_6 pty_country_4 +2014110221 product_11 pty_country_24 +2014110222 product_3 pty_country_10 +2014110222 product_11 pty_country_35 +2014110222 product_1 pty_country_15 +2014110222 product_15 pty_country_13 +2014110222 product_9 pty_country_21 +2014110222 product_1 pty_country_14 +2014110222 product_14 pty_country_8 +2014110222 product_15 pty_country_40 +2014110222 product_14 pty_country_4 +2014110222 product_6 pty_country_20 +2014110223 product_6 pty_country_9 +2014110223 product_5 pty_country_38 +2014110223 product_7 pty_country_5 +2014110223 product_2 pty_country_28 +2014110223 product_3 pty_country_10 +2014110223 product_1 pty_country_2 +2014110223 product_15 pty_country_7 +2014110223 product_6 pty_country_18 +2014110223 product_13 pty_country_45 +2014110223 product_12 pty_country_50 diff --git a/extensions/datasketches/src/test/resources/simple_test_data_aggregators.json b/extensions/datasketches/src/test/resources/simple_test_data_aggregators.json new file mode 100644 index 000000000000..c98b97187871 --- /dev/null +++ b/extensions/datasketches/src/test/resources/simple_test_data_aggregators.json @@ -0,0 +1,12 @@ +[ + { + "type": "thetaSketch", + "name": "pty_country", + "fieldName": "pty_country" + }, + { + "type": "thetaSketch", + "name": "non_existing_col_validation", + "fieldName": "non_existing_col" + } +] diff --git a/extensions/datasketches/src/test/resources/simple_test_data_group_by_query.json b/extensions/datasketches/src/test/resources/simple_test_data_group_by_query.json new file mode 100644 index 000000000000..1b8f80ef02f7 --- /dev/null +++ b/extensions/datasketches/src/test/resources/simple_test_data_group_by_query.json @@ -0,0 +1,93 @@ +{ + "queryType": "groupBy", + "dataSource": "test_datasource", + "granularity": "ALL", + "dimensions": [], + "aggregations": [ + { + "type": "thetaSketch", + "name": "sketch_count", + "fieldName": "pty_country", + "size": 16384 + }, + { + "type": "thetaSketch", + "name": "non_existing_col_validation", + "fieldName": "non_existing_col", + "size": 16384 + } + ], + "postAggregations": [ + { + "type": "thetaSketchEstimate", + "name": "sketchEstimatePostAgg", + "field": { + "type": "fieldAccess", + "fieldName": "sketch_count" + } + }, + { + "type": "thetaSketchEstimate", + "name": "sketchIntersectionPostAggEstimate", + "field": { + "type": "thetaSketchSetOp", + "name": "sketchIntersectionPostAgg", + "func": "INTERSECT", + "size": 16384, + "fields": [ + { + "type": "fieldAccess", + "fieldName": "sketch_count" + }, + { + "type": "fieldAccess", + "fieldName": "sketch_count" + } + ] + } + }, + { + "type": "thetaSketchEstimate", + "name": "sketchAnotBPostAggEstimate", + "field": { + "type": "thetaSketchSetOp", + "name": "sketchAnotBUnionPostAgg", + "func": "NOT", + "size": 16384, + "fields": [ + { + "type": "fieldAccess", + "fieldName": "sketch_count" + }, + { + "type": "fieldAccess", + "fieldName": "sketch_count" + } + ] + } + }, + { + "type": "thetaSketchEstimate", + "name": "sketchUnionPostAggEstimate", + "field": { + "type": "thetaSketchSetOp", + "name": "sketchUnionPostAgg", + "func": "UNION", + "size": 16384, + "fields": [ + { + "type": "fieldAccess", + "fieldName": "sketch_count" + }, + { + "type": "fieldAccess", + "fieldName": "sketch_count" + } + ] + } + } + ], + "intervals": [ + "2014-10-19T00:00:00.000Z/2014-10-22T00:00:00.000Z" + ] +} diff --git a/extensions/datasketches/src/test/resources/simple_test_data_record_parser.json b/extensions/datasketches/src/test/resources/simple_test_data_record_parser.json new file mode 100644 index 000000000000..0c9983af072c --- /dev/null +++ b/extensions/datasketches/src/test/resources/simple_test_data_record_parser.json @@ -0,0 +1,22 @@ +{ + "type": "string", + "parseSpec": { + "format": "tsv", + "timestampSpec": { + "column": "timestamp", + "format": "yyyyMMddHH" + }, + "dimensionsSpec": { + "dimensions": [ + "product" + ], + "dimensionExclusions": [], + "spatialDimensions": [] + }, + "columns": [ + "timestamp", + "product", + "pty_country" + ] + } +} diff --git a/extensions/datasketches/src/test/resources/simple_test_data_record_parser2.json b/extensions/datasketches/src/test/resources/simple_test_data_record_parser2.json new file mode 100644 index 000000000000..6f0ebf1a041e --- /dev/null +++ b/extensions/datasketches/src/test/resources/simple_test_data_record_parser2.json @@ -0,0 +1,23 @@ +{ + "type": "string", + "parseSpec": { + "format": "tsv", + "timestampSpec": { + "column": "timestamp", + "format": "yyyyMMddHH" + }, + "dimensionsSpec": { + "dimensions": [ + "product", + "pty_country" + ], + "dimensionExclusions": [], + "spatialDimensions": [] + }, + "columns": [ + "timestamp", + "product", + "pty_country" + ] + } +} diff --git a/extensions/datasketches/src/test/resources/sketch_test_data.tsv b/extensions/datasketches/src/test/resources/sketch_test_data.tsv new file mode 100644 index 000000000000..8180d80647c6 --- /dev/null +++ b/extensions/datasketches/src/test/resources/sketch_test_data.tsv @@ -0,0 +1,3361 @@ +2014102000 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ +2014102000 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102000 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102000 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102000 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102000 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102000 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102000 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102000 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102000 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102000 product_no +2014102001 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102001 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102001 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102001 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102001 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102001 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102001 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102001 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102001 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102001 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102002 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102002 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102002 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102002 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102002 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102002 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102002 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102002 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102002 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102002 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102003 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102003 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102003 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102003 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102003 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102003 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102003 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102003 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102003 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102003 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102004 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102004 product_14 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102004 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102004 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102004 product_5 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102004 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102004 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102004 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102004 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102004 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102005 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102005 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102005 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102005 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102005 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102005 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102005 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102005 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102005 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102005 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102006 product_7 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102006 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102006 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102006 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102006 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102006 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102006 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102006 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102006 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102006 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102007 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102007 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102007 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102007 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102007 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102007 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102007 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102007 product_8 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102007 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102007 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102008 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102008 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102008 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102008 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102008 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102008 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102008 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102008 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102008 product_7 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102008 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102009 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102009 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102009 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102009 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102009 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102009 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102009 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102009 product_9 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102009 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102009 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102010 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102010 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102010 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102010 product_1 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102010 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102010 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102010 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102010 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102010 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102010 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102011 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102011 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102011 product_1 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102011 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102011 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102011 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102011 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102011 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102011 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102011 product_8 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102012 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102012 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102012 product_6 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102012 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102012 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102012 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102012 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102012 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102012 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102012 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102013 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102013 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102013 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102013 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102013 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102013 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102013 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102013 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102013 product_8 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102013 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102014 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102014 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102014 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102014 product_9 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102014 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102014 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102014 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102014 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102014 product_8 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102014 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102015 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102015 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102015 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102015 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102015 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102015 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102015 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102015 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102015 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102015 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102016 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102016 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102016 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102016 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102016 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102016 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102016 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102016 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102016 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102016 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102017 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102017 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102017 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102017 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102017 product_14 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102017 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102017 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102017 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102017 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102017 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102018 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102018 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102018 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102018 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102018 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102018 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102018 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102018 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102018 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102018 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102019 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102019 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102019 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102019 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102019 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102019 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102019 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102019 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102019 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102019 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102020 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102020 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102020 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102020 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102020 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102020 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102020 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102020 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102020 product_4 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102020 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102021 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102021 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102021 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102021 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102021 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102021 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102021 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102021 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102021 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102021 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102022 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102022 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102022 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102022 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102022 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102022 product_15 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102022 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102022 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102022 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102022 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102023 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102023 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102023 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102023 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102023 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102023 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102023 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102023 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102023 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102023 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102100 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102100 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102100 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102100 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102100 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102100 product_10 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102100 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102100 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102100 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102100 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102101 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102101 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102101 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102101 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102101 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102101 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102101 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102101 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102101 product_13 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102101 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102102 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102102 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102102 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102102 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102102 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102102 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102102 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102102 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102102 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102102 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102103 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102103 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102103 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102103 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102103 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102103 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102103 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102103 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102103 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102103 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102104 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102104 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102104 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102104 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102104 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102104 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102104 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102104 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102104 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102104 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102105 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102105 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102105 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102105 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102105 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102105 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102105 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102105 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102105 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102105 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102106 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102106 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102106 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102106 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102106 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102106 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102106 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102106 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102106 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102106 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102107 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102107 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102107 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102107 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102107 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102107 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102107 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102107 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102107 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102107 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102108 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102108 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102108 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102108 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102108 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102108 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102108 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102108 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102108 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102108 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102109 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102109 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102109 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102109 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102109 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102109 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102109 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102109 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102109 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102109 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102110 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102110 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102110 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102110 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102110 product_10 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102110 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102110 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102110 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102110 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102110 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102111 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102111 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102111 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102111 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102111 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102111 product_14 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102111 product_7 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102111 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102111 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102111 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102112 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102112 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102112 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102112 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102112 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102112 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102112 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102112 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102112 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102112 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102113 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102113 product_9 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102113 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102113 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102113 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102113 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102113 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102113 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102113 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102113 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102114 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102114 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102114 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102114 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102114 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102114 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102114 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102114 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102114 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102114 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102115 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102115 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102115 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102115 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102115 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102115 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102115 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102115 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102115 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102115 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102116 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102116 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102116 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102116 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102116 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102116 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102116 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102116 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102116 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102116 product_6 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102117 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102117 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102117 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102117 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102117 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102117 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102117 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102117 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102117 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102117 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102118 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102118 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102118 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102118 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102118 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102118 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102118 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102118 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102118 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102118 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102119 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102119 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102119 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102119 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102119 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102119 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102119 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102119 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102119 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102119 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102120 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102120 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102120 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102120 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102120 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102120 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102120 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102120 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102120 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102120 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102121 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102121 product_9 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102121 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102121 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102121 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102121 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102121 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102121 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102121 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102121 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102122 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102122 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102122 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102122 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102122 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102122 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102122 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102122 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102122 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102122 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102123 product_10 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102123 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102123 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102123 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102123 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102123 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102123 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102123 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102123 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102123 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102200 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102200 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102200 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102200 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102200 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102200 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102200 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102200 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102200 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102200 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102201 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102201 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102201 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102201 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102201 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102201 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102201 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102201 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102201 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102201 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102202 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102202 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102202 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102202 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102202 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102202 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102202 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102202 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102202 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102202 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102203 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102203 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102203 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102203 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102203 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102203 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102203 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102203 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102203 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102203 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102204 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102204 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102204 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102204 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102204 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102204 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102204 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102204 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102204 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102204 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102205 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102205 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102205 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102205 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102205 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102205 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102205 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102205 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102205 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102205 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102206 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102206 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102206 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102206 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102206 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102206 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102206 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102206 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102206 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102206 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102207 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102207 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102207 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102207 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102207 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102207 product_14 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102207 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102207 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102207 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102207 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102208 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102208 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102208 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102208 product_15 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102208 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102208 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102208 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102208 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102208 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102208 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102209 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102209 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102209 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102209 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102209 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102209 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102209 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102209 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102209 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102209 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102210 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102210 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102210 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102210 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102210 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102210 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102210 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102210 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102210 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102210 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102211 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102211 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102211 product_6 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102211 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102211 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102211 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102211 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102211 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102211 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102211 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102212 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102212 product_7 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102212 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102212 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102212 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102212 product_10 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102212 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102212 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102212 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102212 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102213 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102213 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102213 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102213 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102213 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102213 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102213 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102213 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102213 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102213 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102214 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102214 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102214 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102214 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102214 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102214 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102214 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102214 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102214 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102214 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102215 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102215 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102215 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102215 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102215 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102215 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102215 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102215 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102215 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102215 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102216 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102216 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102216 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102216 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102216 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102216 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102216 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102216 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102216 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102216 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102217 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102217 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102217 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102217 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102217 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102217 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102217 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102217 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102217 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102217 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102218 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102218 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102218 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102218 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102218 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102218 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102218 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102218 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102218 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102218 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102219 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102219 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102219 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102219 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102219 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102219 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102219 product_9 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102219 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102219 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102219 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102220 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102220 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102220 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102220 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102220 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102220 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102220 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102220 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102220 product_2 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102220 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102221 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102221 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102221 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102221 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102221 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102221 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102221 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102221 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102221 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102221 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102222 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102222 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102222 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102222 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102222 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102222 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102222 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102222 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102222 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102222 product_4 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102223 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102223 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102223 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102223 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102223 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102223 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102223 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102223 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102223 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102223 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102300 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102300 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102300 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102300 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102300 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102300 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102300 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102300 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102300 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102300 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102301 product_4 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102301 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102301 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102301 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102301 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102301 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102301 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102301 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102301 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102301 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102302 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102302 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102302 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102302 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102302 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102302 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102302 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102302 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102302 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102302 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102303 product_12 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102303 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102303 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102303 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102303 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102303 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102303 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102303 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102303 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102303 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102304 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102304 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102304 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102304 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102304 product_10 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102304 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102304 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102304 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102304 product_9 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102304 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102305 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102305 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102305 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102305 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102305 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102305 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102305 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102305 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102305 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102305 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102306 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102306 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102306 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102306 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102306 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102306 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102306 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102306 product_2 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102306 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102306 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102307 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102307 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102307 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102307 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102307 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102307 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102307 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102307 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102307 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102307 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102308 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102308 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102308 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102308 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102308 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102308 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102308 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102308 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102308 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102308 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102309 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102309 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102309 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102309 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102309 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102309 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102309 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102309 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102309 product_10 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102309 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102310 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102310 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102310 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102310 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102310 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102310 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102310 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102310 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102310 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102310 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102311 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102311 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102311 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102311 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102311 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102311 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102311 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102311 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102311 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102311 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102312 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102312 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102312 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102312 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102312 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102312 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102312 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102312 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102312 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102312 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102313 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102313 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102313 product_11 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102313 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102313 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102313 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102313 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102313 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102313 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102313 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102314 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102314 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102314 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102314 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102314 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102314 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102314 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102314 product_2 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102314 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102314 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102315 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102315 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102315 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102315 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102315 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102315 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102315 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102315 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102315 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102315 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102316 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102316 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102316 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102316 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102316 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102316 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102316 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102316 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102316 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102316 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102317 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102317 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102317 product_13 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102317 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102317 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102317 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102317 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102317 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102317 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102317 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102318 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102318 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102318 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102318 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102318 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102318 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102318 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102318 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102318 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102318 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102319 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102319 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102319 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102319 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102319 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102319 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102319 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102319 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102319 product_14 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102319 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102320 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102320 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102320 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102320 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102320 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102320 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102320 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102320 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102320 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102320 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102321 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102321 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102321 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102321 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102321 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102321 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102321 product_4 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102321 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102321 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102321 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102322 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102322 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102322 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102322 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102322 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102322 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102322 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102322 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102322 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102322 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102323 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102323 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102323 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102323 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102323 product_5 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102323 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102323 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102323 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102323 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102323 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102400 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102400 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102400 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102400 product_12 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102400 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102400 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102400 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102400 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102400 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102400 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102401 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102401 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102401 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102401 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102401 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102401 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102401 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102401 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102401 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102401 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102402 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102402 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102402 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102402 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102402 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102402 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102402 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102402 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102402 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102402 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102403 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102403 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102403 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102403 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102403 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102403 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102403 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102403 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102403 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102403 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102404 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102404 product_10 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102404 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102404 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102404 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102404 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102404 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102404 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102404 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102404 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102405 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102405 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102405 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102405 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102405 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102405 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102405 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102405 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102405 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102405 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102406 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102406 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102406 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102406 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102406 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102406 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102406 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102406 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102406 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102406 product_4 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102407 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102407 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102407 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102407 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102407 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102407 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102407 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102407 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102407 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102407 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102408 product_8 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102408 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102408 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102408 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102408 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102408 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102408 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102408 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102408 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102408 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102409 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102409 product_9 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102409 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102409 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102409 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102409 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102409 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102409 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102409 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102409 product_14 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102410 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102410 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102410 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102410 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102410 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102410 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102410 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102410 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102410 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102410 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102411 product_9 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102411 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102411 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102411 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102411 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102411 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102411 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102411 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102411 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102411 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102412 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102412 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102412 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102412 product_1 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102412 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102412 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102412 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102412 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102412 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102412 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102413 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102413 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102413 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102413 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102413 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102413 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102413 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102413 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102413 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102413 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102414 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102414 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102414 product_7 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102414 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102414 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102414 product_10 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102414 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102414 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102414 product_5 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102414 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102415 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102415 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102415 product_3 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102415 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102415 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102415 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102415 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102415 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102415 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102415 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102416 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102416 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102416 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102416 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102416 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102416 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102416 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102416 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102416 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102416 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102417 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102417 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102417 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102417 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102417 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102417 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102417 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102417 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102417 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102417 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102418 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102418 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102418 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102418 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102418 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102418 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102418 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102418 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102418 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102418 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102419 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102419 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102419 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102419 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102419 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102419 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102419 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102419 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102419 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102419 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102420 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102420 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102420 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102420 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102420 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102420 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102420 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102420 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102420 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102420 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102421 product_8 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102421 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102421 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102421 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102421 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102421 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102421 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102421 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102421 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102421 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102422 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102422 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102422 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102422 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102422 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102422 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102422 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102422 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102422 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102422 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102423 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102423 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102423 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102423 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102423 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102423 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102423 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102423 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102423 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102423 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102500 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102500 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102500 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102500 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102500 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102500 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102500 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102500 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102500 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102500 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102501 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102501 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102501 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102501 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102501 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102501 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102501 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102501 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102501 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102501 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102502 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102502 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102502 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102502 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102502 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102502 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102502 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102502 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102502 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102502 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102503 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102503 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102503 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102503 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102503 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102503 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102503 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102503 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102503 product_5 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102503 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102504 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102504 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102504 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102504 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102504 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102504 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102504 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102504 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102504 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102504 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102505 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102505 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102505 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102505 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102505 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102505 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102505 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102505 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102505 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102505 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102506 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102506 product_10 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102506 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102506 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102506 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102506 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102506 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102506 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102506 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102506 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102507 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102507 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102507 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102507 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102507 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102507 product_6 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102507 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102507 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102507 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102507 product_3 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102508 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102508 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102508 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102508 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102508 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102508 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102508 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102508 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102508 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102508 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102509 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102509 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102509 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102509 product_15 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102509 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102509 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102509 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102509 product_2 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102509 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102509 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102510 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102510 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102510 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102510 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102510 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102510 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102510 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102510 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102510 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102510 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102511 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102511 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102511 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102511 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102511 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102511 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102511 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102511 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102511 product_12 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102511 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102512 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102512 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102512 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102512 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102512 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102512 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102512 product_2 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102512 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102512 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102512 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102513 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102513 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102513 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102513 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102513 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102513 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102513 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102513 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102513 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102513 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102514 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102514 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102514 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102514 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102514 product_3 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102514 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102514 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102514 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102514 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102514 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102515 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102515 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102515 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102515 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102515 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102515 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102515 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102515 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102515 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102515 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102516 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102516 product_10 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102516 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102516 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102516 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102516 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102516 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102516 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102516 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102516 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102517 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102517 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102517 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102517 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102517 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102517 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102517 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102517 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102517 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102517 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102518 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102518 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102518 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102518 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102518 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102518 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102518 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102518 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102518 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102518 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102519 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102519 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102519 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102519 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102519 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102519 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102519 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102519 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102519 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102519 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102520 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102520 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102520 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102520 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102520 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102520 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102520 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102520 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102520 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102520 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102521 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102521 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102521 product_3 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102521 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102521 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102521 product_13 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102521 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102521 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102521 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102521 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102522 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102522 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102522 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102522 product_7 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102522 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102522 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102522 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102522 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102522 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102522 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102523 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102523 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102523 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102523 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102523 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102523 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102523 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102523 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102523 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102523 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102600 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102600 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102600 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102600 product_1 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102600 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102600 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102600 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102600 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102600 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102600 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102601 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102601 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102601 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102601 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102601 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102601 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102601 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102601 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102601 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102601 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102602 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102602 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102602 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102602 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102602 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102602 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102602 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102602 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102602 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102602 product_12 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102603 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102603 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102603 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102603 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102603 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102603 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102603 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102603 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102603 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102603 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102604 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102604 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102604 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102604 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102604 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102604 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102604 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102604 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102604 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102604 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102605 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102605 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102605 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102605 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102605 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102605 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102605 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102605 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102605 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102605 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102606 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102606 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102606 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102606 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102606 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102606 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102606 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102606 product_10 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102606 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102606 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102607 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102607 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102607 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102607 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102607 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102607 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102607 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102607 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102607 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102607 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102608 product_15 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102608 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102608 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102608 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102608 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102608 product_6 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102608 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102608 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102608 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102608 product_9 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102609 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102609 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102609 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102609 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102609 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102609 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102609 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102609 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102609 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102609 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102610 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102610 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102610 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102610 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102610 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102610 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102610 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102610 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102610 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102610 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102611 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102611 product_11 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102611 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102611 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102611 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102611 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102611 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102611 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102611 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102611 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102612 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102612 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102612 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102612 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102612 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102612 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102612 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102612 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102612 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102612 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102613 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102613 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102613 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102613 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102613 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102613 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102613 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102613 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102613 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102613 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102614 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102614 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102614 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102614 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102614 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102614 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102614 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102614 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102614 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102614 product_13 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102615 product_5 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102615 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102615 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102615 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102615 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102615 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102615 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102615 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102615 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102615 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102616 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102616 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102616 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102616 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102616 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102616 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102616 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102616 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102616 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102616 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102617 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102617 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102617 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102617 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102617 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102617 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102617 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102617 product_8 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102617 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102617 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102618 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102618 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102618 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102618 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102618 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102618 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102618 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102618 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102618 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102618 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102619 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102619 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102619 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102619 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102619 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102619 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102619 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102619 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102619 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102619 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102620 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102620 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102620 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102620 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102620 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102620 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102620 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102620 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102620 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102620 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102621 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102621 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102621 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102621 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102621 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102621 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102621 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102621 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102621 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102621 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102622 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102622 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102622 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102622 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102622 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102622 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102622 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102622 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102622 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102622 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102623 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102623 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102623 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102623 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102623 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102623 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102623 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102623 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102623 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102623 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102700 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102700 product_4 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102700 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102700 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102700 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102700 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102700 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102700 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102700 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102700 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102701 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102701 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102701 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102701 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102701 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102701 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102701 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102701 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102701 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102701 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102702 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102702 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102702 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102702 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102702 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102702 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102702 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102702 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102702 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102702 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102703 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102703 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102703 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102703 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102703 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102703 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102703 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102703 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102703 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102703 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102704 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102704 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102704 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102704 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102704 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102704 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102704 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102704 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102704 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102704 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102705 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102705 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102705 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102705 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102705 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102705 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102705 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102705 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102705 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102705 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102706 product_10 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102706 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102706 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102706 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102706 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102706 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102706 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102706 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102706 product_14 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102706 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102707 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102707 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102707 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102707 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102707 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102707 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102707 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102707 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102707 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102707 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102708 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102708 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102708 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102708 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102708 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102708 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102708 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102708 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102708 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102708 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102709 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102709 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102709 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102709 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102709 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102709 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102709 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102709 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102709 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102709 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102710 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102710 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102710 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102710 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102710 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102710 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102710 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102710 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102710 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102710 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102711 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102711 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102711 product_7 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102711 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102711 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102711 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102711 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102711 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102711 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102711 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102712 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102712 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102712 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102712 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102712 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102712 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102712 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102712 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102712 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102712 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102713 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102713 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102713 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102713 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102713 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102713 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102713 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102713 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102713 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102713 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102714 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102714 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102714 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102714 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102714 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102714 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102714 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102714 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102714 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102714 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102715 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102715 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102715 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102715 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102715 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102715 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102715 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102715 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102715 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102715 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102716 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102716 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102716 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102716 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102716 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102716 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102716 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102716 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102716 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102716 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102717 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102717 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102717 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102717 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102717 product_13 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102717 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102717 product_5 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102717 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102717 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102717 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102718 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102718 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102718 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102718 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102718 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102718 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102718 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102718 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102718 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102718 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102719 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102719 product_9 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102719 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102719 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102719 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102719 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102719 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102719 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102719 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102719 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102720 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102720 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102720 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102720 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102720 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102720 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102720 product_11 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102720 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102720 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102720 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102721 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102721 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102721 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102721 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102721 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102721 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102721 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102721 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102721 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102721 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102722 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102722 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102722 product_14 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102722 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102722 product_10 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102722 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102722 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102722 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102722 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102722 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102723 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102723 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102723 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102723 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102723 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102723 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102723 product_3 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102723 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102723 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102723 product_14 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102800 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102800 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102800 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102800 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102800 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102800 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102800 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102800 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102800 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102800 product_13 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102801 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102801 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102801 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102801 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102801 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102801 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102801 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102801 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102801 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102801 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102802 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102802 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102802 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102802 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102802 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102802 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102802 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102802 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102802 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102802 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102803 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102803 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102803 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102803 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102803 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102803 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102803 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102803 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102803 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102803 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102804 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102804 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102804 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102804 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102804 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102804 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102804 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102804 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102804 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102804 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102805 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102805 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102805 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102805 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102805 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102805 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102805 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102805 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102805 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102805 product_13 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102806 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102806 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102806 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102806 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102806 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102806 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102806 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102806 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102806 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102806 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102807 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102807 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102807 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102807 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102807 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102807 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102807 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102807 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102807 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102807 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102808 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102808 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102808 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102808 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102808 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102808 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102808 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102808 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102808 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102808 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102809 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102809 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102809 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102809 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102809 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102809 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102809 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102809 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102809 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102809 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102810 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102810 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102810 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102810 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102810 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102810 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102810 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102810 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102810 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102810 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102811 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102811 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102811 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102811 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102811 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102811 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102811 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102811 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102811 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102811 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102812 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102812 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102812 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102812 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102812 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102812 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102812 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102812 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102812 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102812 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102813 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102813 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102813 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102813 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102813 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102813 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102813 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102813 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102813 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102813 product_12 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102814 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102814 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102814 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102814 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102814 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102814 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102814 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102814 product_6 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102814 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102814 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102815 product_7 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102815 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102815 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102815 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102815 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102815 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102815 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102815 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102815 product_6 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102815 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102816 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102816 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102816 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102816 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102816 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102816 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102816 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102816 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102816 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102816 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102817 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102817 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102817 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102817 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102817 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102817 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102817 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102817 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102817 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102817 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102818 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102818 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102818 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102818 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102818 product_4 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102818 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102818 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102818 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102818 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102818 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102819 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102819 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102819 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102819 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102819 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102819 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102819 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102819 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102819 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102819 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102820 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102820 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102820 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102820 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102820 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102820 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102820 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102820 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102820 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102820 product_6 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102821 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102821 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102821 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102821 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102821 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102821 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102821 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102821 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102821 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102821 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102822 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102822 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102822 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102822 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102822 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102822 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102822 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102822 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102822 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102822 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102823 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102823 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102823 product_11 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102823 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102823 product_12 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102823 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102823 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102823 product_15 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102823 product_5 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102823 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102900 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102900 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102900 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102900 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102900 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102900 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102900 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102900 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102900 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102900 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102901 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102901 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102901 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102901 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102901 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102901 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102901 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102901 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102901 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102901 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102902 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102902 product_1 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102902 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102902 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102902 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102902 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102902 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102902 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102902 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102902 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102903 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102903 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102903 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102903 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102903 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102903 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102903 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102903 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102903 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102903 product_1 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102904 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102904 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102904 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102904 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102904 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102904 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102904 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102904 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102904 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102904 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102905 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102905 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102905 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102905 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102905 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102905 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102905 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102905 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102905 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102905 product_8 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102906 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102906 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102906 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102906 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102906 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102906 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102906 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102906 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102906 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102906 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102907 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102907 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102907 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102907 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102907 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102907 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102907 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102907 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102907 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102907 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102908 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102908 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102908 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102908 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102908 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102908 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102908 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102908 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102908 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102908 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102909 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102909 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102909 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102909 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102909 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102909 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102909 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102909 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102909 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102909 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102910 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102910 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102910 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102910 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102910 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102910 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102910 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102910 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102910 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102910 product_5 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102911 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102911 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102911 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102911 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102911 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102911 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102911 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102911 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102911 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102911 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102912 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102912 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102912 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102912 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102912 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102912 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102912 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102912 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102912 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102912 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102913 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102913 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102913 product_3 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102913 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102913 product_12 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102913 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102913 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102913 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102913 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102913 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102914 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102914 product_2 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102914 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102914 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102914 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102914 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102914 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102914 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102914 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102914 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102915 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102915 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102915 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102915 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102915 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102915 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102915 product_6 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102915 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102915 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102915 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102916 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102916 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102916 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102916 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102916 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102916 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102916 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102916 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102916 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102916 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102917 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102917 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102917 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102917 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102917 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102917 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102917 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102917 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102917 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102917 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102918 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102918 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102918 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102918 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102918 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102918 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102918 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102918 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102918 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102918 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102919 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102919 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102919 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102919 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102919 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102919 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102919 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102919 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102919 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102919 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102920 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102920 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102920 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102920 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102920 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102920 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102920 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102920 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102920 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102920 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102921 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102921 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102921 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102921 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102921 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102921 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102921 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102921 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102921 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102921 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102922 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102922 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102922 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102922 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102922 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102922 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102922 product_4 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102922 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102922 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102922 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102923 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102923 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102923 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102923 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102923 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102923 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102923 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102923 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102923 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102923 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014103000 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014103000 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014103000 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014103000 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014103000 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014103000 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014103000 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014103000 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014103000 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014103000 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014103001 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014103001 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014103001 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014103001 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014103001 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014103001 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014103001 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014103001 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014103001 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014103001 product_7 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014103002 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014103002 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014103002 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014103002 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014103002 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014103002 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014103002 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014103002 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014103002 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014103002 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014103003 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014103003 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014103003 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014103003 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014103003 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014103003 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014103003 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014103003 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014103003 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014103003 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014103004 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014103004 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014103004 product_11 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014103004 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014103004 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014103004 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014103004 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014103004 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014103004 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014103004 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014103005 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014103005 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014103005 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014103005 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014103005 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014103005 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014103005 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014103005 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014103005 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014103005 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014103006 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014103006 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014103006 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014103006 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014103006 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014103006 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014103006 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014103006 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014103006 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014103006 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014103007 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014103007 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014103007 product_7 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014103007 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014103007 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014103007 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014103007 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014103007 product_3 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014103007 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014103007 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014103008 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014103008 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014103008 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014103008 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014103008 product_14 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014103008 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014103008 product_10 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014103008 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014103008 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014103008 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014103009 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014103009 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014103009 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014103009 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014103009 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014103009 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014103009 product_7 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014103009 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014103009 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014103009 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014103010 product_7 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014103010 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014103010 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014103010 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014103010 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014103010 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014103010 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014103010 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014103010 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014103010 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014103011 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014103011 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014103011 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014103011 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014103011 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014103011 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014103011 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014103011 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014103011 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014103011 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014103012 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014103012 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014103012 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014103012 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014103012 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014103012 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014103012 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014103012 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014103012 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014103012 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014103013 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014103013 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014103013 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014103013 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014103013 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014103013 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014103013 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014103013 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014103013 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014103013 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014103014 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014103014 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014103014 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014103014 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014103014 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014103014 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014103014 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014103014 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014103014 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014103014 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014103015 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014103015 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014103015 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014103015 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014103015 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014103015 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014103015 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014103015 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014103015 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014103015 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014103016 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014103016 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014103016 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014103016 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014103016 product_1 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014103016 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014103016 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014103016 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014103016 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014103016 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014103017 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014103017 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014103017 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014103017 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014103017 product_2 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014103017 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014103017 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014103017 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014103017 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014103017 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014103018 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014103018 product_3 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014103018 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014103018 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014103018 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014103018 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014103018 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014103018 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014103018 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014103018 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014103019 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014103019 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014103019 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014103019 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014103019 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014103019 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014103019 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014103019 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014103019 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014103019 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014103020 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014103020 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014103020 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014103020 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014103020 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014103020 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014103020 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014103020 product_1 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014103020 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014103020 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014103021 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014103021 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014103021 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014103021 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014103021 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014103021 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014103021 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014103021 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014103021 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014103021 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014103022 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014103022 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014103022 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014103022 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014103022 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014103022 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014103022 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014103022 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014103022 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014103022 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014103023 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014103023 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014103023 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014103023 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014103023 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014103023 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014103023 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014103023 product_8 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014103023 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014103023 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014103100 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014103100 product_8 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014103100 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014103100 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014103100 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014103100 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014103100 product_7 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014103100 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014103100 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014103100 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014103101 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014103101 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014103101 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014103101 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014103101 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014103101 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014103101 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014103101 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014103101 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014103101 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014103102 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014103102 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014103102 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014103102 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014103102 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014103102 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014103102 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014103102 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014103102 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014103102 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014103103 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014103103 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014103103 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014103103 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014103103 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014103103 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014103103 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014103103 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014103103 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014103103 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014103104 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014103104 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014103104 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014103104 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014103104 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014103104 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014103104 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014103104 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014103104 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014103104 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014103105 product_14 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014103105 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014103105 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014103105 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014103105 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014103105 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014103105 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014103105 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014103105 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014103105 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014103106 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014103106 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014103106 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014103106 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014103106 product_14 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014103106 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014103106 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014103106 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014103106 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014103106 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014103107 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014103107 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014103107 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014103107 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014103107 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014103107 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014103107 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014103107 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014103107 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014103107 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014103108 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014103108 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014103108 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014103108 product_10 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014103108 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014103108 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014103108 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014103108 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014103108 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014103108 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014103109 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014103109 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014103109 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014103109 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014103109 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014103109 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014103109 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014103109 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014103109 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014103109 product_11 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014103110 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014103110 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014103110 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014103110 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014103110 product_14 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014103110 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014103110 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014103110 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014103110 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014103110 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014103111 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014103111 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014103111 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014103111 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014103111 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014103111 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014103111 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014103111 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014103111 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014103111 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014103112 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014103112 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014103112 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014103112 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014103112 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014103112 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014103112 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014103112 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014103112 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014103112 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014103113 product_5 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014103113 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014103113 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014103113 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014103113 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014103113 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014103113 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014103113 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014103113 product_15 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014103113 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014103114 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014103114 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014103114 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014103114 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014103114 product_4 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014103114 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014103114 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014103114 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014103114 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014103114 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014103115 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014103115 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014103115 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014103115 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014103115 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014103115 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014103115 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014103115 product_13 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014103115 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014103115 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014103116 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014103116 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014103116 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014103116 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014103116 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014103116 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014103116 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014103116 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014103116 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014103116 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014103117 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014103117 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014103117 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014103117 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014103117 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014103117 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014103117 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014103117 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014103117 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014103117 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014103118 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014103118 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014103118 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014103118 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014103118 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014103118 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014103118 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014103118 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014103118 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014103118 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014103119 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014103119 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014103119 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014103119 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014103119 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014103119 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014103119 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014103119 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014103119 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014103119 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014103120 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014103120 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014103120 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014103120 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014103120 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014103120 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014103120 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014103120 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014103120 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014103120 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014103121 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014103121 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014103121 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014103121 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014103121 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014103121 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014103121 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014103121 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014103121 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014103121 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014103122 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014103122 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014103122 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014103122 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014103122 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014103122 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014103122 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014103122 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014103122 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014103122 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014103123 product_7 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014103123 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014103123 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014103123 product_5 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014103123 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014103123 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014103123 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014103123 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014103123 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014103123 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014110100 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014110100 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014110100 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014110100 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014110100 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014110100 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014110100 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014110100 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014110100 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014110100 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014110101 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014110101 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014110101 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014110101 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014110101 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014110101 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014110101 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014110101 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014110101 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014110101 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014110102 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014110102 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014110102 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014110102 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014110102 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014110102 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014110102 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014110102 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014110102 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014110102 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014110103 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014110103 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014110103 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014110103 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014110103 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014110103 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014110103 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014110103 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014110103 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014110103 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014110104 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014110104 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014110104 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014110104 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014110104 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014110104 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014110104 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014110104 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014110104 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014110104 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014110105 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014110105 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014110105 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014110105 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014110105 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014110105 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014110105 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014110105 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014110105 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014110105 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014110106 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014110106 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014110106 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014110106 product_12 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014110106 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014110106 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014110106 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014110106 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014110106 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014110106 product_15 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014110107 product_11 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014110107 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014110107 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014110107 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014110107 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014110107 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014110107 product_7 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014110107 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014110107 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014110107 product_1 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014110108 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014110108 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014110108 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014110108 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014110108 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014110108 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014110108 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014110108 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014110108 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014110108 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014110109 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014110109 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014110109 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014110109 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014110109 product_4 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014110109 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014110109 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014110109 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014110109 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014110109 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014110110 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014110110 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014110110 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014110110 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014110110 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014110110 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014110110 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014110110 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014110110 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014110110 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014110111 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014110111 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014110111 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014110111 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014110111 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014110111 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014110111 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014110111 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014110111 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014110111 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014110112 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014110112 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014110112 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014110112 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014110112 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014110112 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014110112 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014110112 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014110112 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014110112 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014110113 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014110113 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014110113 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014110113 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014110113 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014110113 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014110113 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014110113 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014110113 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014110113 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014110114 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014110114 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014110114 product_2 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014110114 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014110114 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014110114 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014110114 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014110114 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014110114 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014110114 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014110115 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014110115 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014110115 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014110115 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014110115 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014110115 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014110115 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014110115 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014110115 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014110115 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014110116 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014110116 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014110116 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014110116 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014110116 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014110116 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014110116 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014110116 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014110116 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014110116 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014110117 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014110117 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014110117 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014110117 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014110117 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014110117 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014110117 product_1 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014110117 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014110117 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014110117 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014110118 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014110118 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014110118 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014110118 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014110118 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014110118 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014110118 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014110118 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014110118 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014110118 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014110119 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014110119 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014110119 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014110119 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014110119 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014110119 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014110119 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014110119 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014110119 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014110119 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014110120 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014110120 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014110120 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014110120 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014110120 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014110120 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014110120 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014110120 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014110120 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014110120 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014110121 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014110121 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014110121 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014110121 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014110121 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014110121 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014110121 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014110121 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014110121 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014110121 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014110122 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014110122 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014110122 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014110122 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014110122 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014110122 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014110122 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014110122 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014110122 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014110122 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014110123 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014110123 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014110123 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014110123 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014110123 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014110123 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014110123 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014110123 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014110123 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014110123 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014110200 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014110200 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014110200 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014110200 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014110200 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014110200 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014110200 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014110200 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014110200 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014110200 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014110201 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014110201 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014110201 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014110201 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014110201 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014110201 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014110201 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014110201 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014110201 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014110201 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014110202 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014110202 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014110202 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014110202 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014110202 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014110202 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014110202 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014110202 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014110202 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014110202 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014110203 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014110203 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014110203 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014110203 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014110203 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014110203 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014110203 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014110203 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014110203 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014110203 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014110204 product_13 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014110204 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014110204 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014110204 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014110204 product_6 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014110204 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014110204 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014110204 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014110204 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014110204 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014110205 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014110205 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014110205 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014110205 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014110205 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014110205 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014110205 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014110205 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014110205 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014110205 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014110206 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014110206 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014110206 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014110206 product_12 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014110206 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014110206 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014110206 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014110206 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014110206 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014110206 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014110207 product_10 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014110207 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014110207 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014110207 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014110207 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014110207 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014110207 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014110207 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014110207 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014110207 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014110208 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014110208 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014110208 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014110208 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014110208 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014110208 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014110208 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014110208 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014110208 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014110208 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014110209 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014110209 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014110209 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014110209 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014110209 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014110209 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014110209 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014110209 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014110209 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014110209 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014110210 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014110210 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014110210 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014110210 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014110210 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014110210 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014110210 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014110210 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014110210 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014110210 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014110211 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014110211 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014110211 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014110211 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014110211 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014110211 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014110211 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014110211 product_2 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014110211 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014110211 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014110212 product_4 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014110212 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014110212 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014110212 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014110212 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014110212 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014110212 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014110212 product_3 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014110212 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014110212 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014110213 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014110213 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014110213 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014110213 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014110213 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014110213 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014110213 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014110213 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014110213 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014110213 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014110214 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014110214 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014110214 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014110214 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014110214 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014110214 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014110214 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014110214 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014110214 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014110214 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014110215 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014110215 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014110215 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014110215 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014110215 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014110215 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014110215 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014110215 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014110215 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014110215 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014110216 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014110216 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014110216 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014110216 product_5 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014110216 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014110216 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014110216 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014110216 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014110216 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014110216 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014110217 product_12 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014110217 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014110217 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014110217 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014110217 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014110217 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014110217 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014110217 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014110217 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014110217 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014110218 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014110218 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014110218 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014110218 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014110218 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014110218 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014110218 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014110218 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014110218 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014110218 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014110219 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014110219 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014110219 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014110219 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014110219 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014110219 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014110219 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014110219 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014110219 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014110219 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014110220 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014110220 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014110220 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014110220 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014110220 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014110220 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014110220 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014110220 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014110220 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014110220 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014110221 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014110221 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014110221 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014110221 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014110221 product_1 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014110221 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014110221 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014110221 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014110221 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014110221 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014110222 product_3 AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014110222 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014110222 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014110222 product_15 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014110222 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014110222 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014110222 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014110222 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014110222 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014110222 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014110223 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014110223 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014110223 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014110223 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014110223 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014110223 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014110223 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014110223 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014110223 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014110223 product_12 AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= diff --git a/extensions/datasketches/src/test/resources/sketch_test_data_aggregators.json b/extensions/datasketches/src/test/resources/sketch_test_data_aggregators.json new file mode 100644 index 000000000000..05cd0baa59bb --- /dev/null +++ b/extensions/datasketches/src/test/resources/sketch_test_data_aggregators.json @@ -0,0 +1,16 @@ +[ + { + "type": "thetaSketch", + "name": "sids_sketch", + "fieldName": "sketch", + "isInputThetaSketch": true, + "size": 16384 + }, + { + "type": "thetaSketch", + "name": "non_existing_col_validation", + "fieldName": "non_existing_col", + "isInputThetaSketch": true, + "size": 16384 + } +] diff --git a/extensions/datasketches/src/test/resources/sketch_test_data_group_by_query.json b/extensions/datasketches/src/test/resources/sketch_test_data_group_by_query.json new file mode 100644 index 000000000000..d4d908a2f28f --- /dev/null +++ b/extensions/datasketches/src/test/resources/sketch_test_data_group_by_query.json @@ -0,0 +1,92 @@ +{ + "queryType": "groupBy", + "dataSource": "test_datasource", + "granularity": "ALL", + "dimensions": [], + "aggregations": [ + { + "type": "thetaSketch", + "name": "sids_sketch_count", + "fieldName": "sids_sketch", + "size": 16384 + }, + { + "type": "thetaSketch", + "name": "non_existing_col_validation", + "fieldName": "non_existing_col", + "size": 16384 + } + ], + "postAggregations": [ + { + "type": "thetaSketchEstimate", + "name": "sketchEstimatePostAgg", + "field": { + "type": "fieldAccess", + "fieldName": "sids_sketch_count" + } + }, + { + "type": "thetaSketchEstimate", + "name": "sketchIntersectionPostAggEstimate", + "field": { + "type": "thetaSketchSetOp", + "name": "sketchIntersectionPostAgg", + "func": "INTERSECT", + "size": 16384, + "fields": [ + { + "type": "fieldAccess", + "fieldName": "sids_sketch_count" + }, + { + "type": "fieldAccess", + "fieldName": "sids_sketch_count" + } + ] + } + }, + { + "type": "thetaSketchEstimate", + "name": "sketchAnotBPostAggEstimate", + "field": { + "type": "thetaSketchSetOp", + "name": "sketchAnotBUnionPostAgg", + "func": "NOT", + "fields": [ + { + "type": "fieldAccess", + "fieldName": "sids_sketch_count" + }, + { + "type": "fieldAccess", + "fieldName": "sids_sketch_count" + } + ] + } + }, + { + "type": "thetaSketchEstimate", + "name": "sketchUnionPostAggEstimate", + "field": { + "type": "thetaSketchSetOp", + "name": "sketchUnionPostAgg", + "func": "UNION", + "size": 16384, + "fields": [ + { + "type": "fieldAccess", + "fieldName": "sids_sketch_count" + }, + { + "type": "fieldAccess", + "fieldName": "sids_sketch_count" + } + ] + } + } + ], + "intervals": [ + "2014-10-19T00:00:00.000Z/2014-10-22T00:00:00.000Z" + ] +} diff --git a/extensions/datasketches/src/test/resources/sketch_test_data_record_parser.json b/extensions/datasketches/src/test/resources/sketch_test_data_record_parser.json new file mode 100644 index 000000000000..156c5d622267 --- /dev/null +++ b/extensions/datasketches/src/test/resources/sketch_test_data_record_parser.json @@ -0,0 +1,22 @@ +{ + "type": "string", + "parseSpec": { + "format": "tsv", + "timestampSpec": { + "column": "timestamp", + "format": "yyyyMMddHH" + }, + "dimensionsSpec": { + "dimensions": [ + "product" + ], + "dimensionExclusions": [], + "spatialDimensions": [] + }, + "columns": [ + "timestamp", + "product", + "sketch" + ] + } +} diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 70f162815d7f..bb141e48cf74 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -108,27 +108,20 @@ org.apache.hadoop hadoop-hdfs ${hadoop.compile.version} - test-jar + tests test org.apache.hadoop - hadoop-mapreduce-client-jobclient + hadoop-common ${hadoop.compile.version} - test-jar + tests test org.apache.hadoop - hadoop-minicluster - ${hadoop.compile.version} - test - - - org.apache.hadoop - hadoop-yarn-server-tests + hadoop-hdfs ${hadoop.compile.version} - test-jar test diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/DetermineHashedPartitionsJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/DetermineHashedPartitionsJob.java index 829a03a68441..594841f5201f 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/DetermineHashedPartitionsJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/DetermineHashedPartitionsJob.java @@ -102,7 +102,11 @@ public boolean run() } else { groupByJob.setNumReduceTasks(config.getSegmentGranularIntervals().get().size()); } - JobHelper.setupClasspath(JobHelper.distributedClassPath(config.getWorkingPath()), groupByJob); + JobHelper.setupClasspath( + JobHelper.distributedClassPath(config.getWorkingPath()), + JobHelper.distributedClassPath(config.makeIntermediatePath()), + groupByJob + ); config.addInputPaths(groupByJob); config.intoConfiguration(groupByJob); diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/DeterminePartitionsJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/DeterminePartitionsJob.java index a502c340c1b0..0fe4bb9c5106 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/DeterminePartitionsJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/DeterminePartitionsJob.java @@ -136,7 +136,11 @@ public boolean run() groupByJob.setOutputKeyClass(BytesWritable.class); groupByJob.setOutputValueClass(NullWritable.class); groupByJob.setOutputFormatClass(SequenceFileOutputFormat.class); - JobHelper.setupClasspath(JobHelper.distributedClassPath(config.getWorkingPath()), groupByJob); + JobHelper.setupClasspath( + JobHelper.distributedClassPath(config.getWorkingPath()), + JobHelper.distributedClassPath(config.makeIntermediatePath()), + groupByJob + ); config.addInputPaths(groupByJob); config.intoConfiguration(groupByJob); @@ -186,7 +190,11 @@ public boolean run() dimSelectionJob.setOutputFormatClass(DeterminePartitionsDimSelectionOutputFormat.class); dimSelectionJob.setPartitionerClass(DeterminePartitionsDimSelectionPartitioner.class); dimSelectionJob.setNumReduceTasks(config.getGranularitySpec().bucketIntervals().get().size()); - JobHelper.setupClasspath(JobHelper.distributedClassPath(config.getWorkingPath()), dimSelectionJob); + JobHelper.setupClasspath( + JobHelper.distributedClassPath(config.getWorkingPath()), + JobHelper.distributedClassPath(config.makeIntermediatePath()), + dimSelectionJob + ); config.intoConfiguration(dimSelectionJob); FileOutputFormat.setOutputPath(dimSelectionJob, config.makeIntermediatePath()); diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java index 16de192049f3..e137d8bae642 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java @@ -190,7 +190,11 @@ public boolean run() config.intoConfiguration(job); - JobHelper.setupClasspath(JobHelper.distributedClassPath(config.getWorkingPath()), job); + JobHelper.setupClasspath( + JobHelper.distributedClassPath(config.getWorkingPath()), + JobHelper.distributedClassPath(config.makeIntermediatePath()), + job + ); job.submit(); log.info("Job %s submitted, status available at %s", job.getJobName(), job.getTrackingURL()); diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/JobHelper.java b/indexing-hadoop/src/main/java/io/druid/indexer/JobHelper.java index a9c282ddfadd..dc9c6e53c572 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/JobHelper.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/JobHelper.java @@ -18,6 +18,7 @@ package io.druid.indexer; import com.fasterxml.jackson.core.JsonProcessingException; +import com.google.common.base.Predicate; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Sets; @@ -33,22 +34,6 @@ import io.druid.segment.ProgressIndicator; import io.druid.segment.SegmentUtils; import io.druid.timeline.DataSegment; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.filecache.DistributedCache; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.LocalFileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.retry.RetryPolicies; -import org.apache.hadoop.io.retry.RetryProxy; -import org.apache.hadoop.mapreduce.Job; -import org.apache.hadoop.mapreduce.TaskAttemptContext; -import org.apache.hadoop.mapreduce.TaskAttemptID; -import org.apache.hadoop.util.Progressable; -import org.joda.time.DateTime; -import org.joda.time.Interval; -import org.joda.time.format.ISODateTimeFormat; - import java.io.BufferedOutputStream; import java.io.File; import java.io.FileInputStream; @@ -64,9 +49,24 @@ import java.util.concurrent.Callable; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; +import java.util.regex.Pattern; import java.util.zip.ZipEntry; import java.util.zip.ZipInputStream; import java.util.zip.ZipOutputStream; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocalFileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.retry.RetryPolicies; +import org.apache.hadoop.io.retry.RetryProxy; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.mapreduce.TaskAttemptID; +import org.apache.hadoop.util.Progressable; +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.joda.time.format.ISODateTimeFormat; /** */ @@ -79,6 +79,7 @@ public class JobHelper private static final int NUM_RETRIES = 8; private static final int SECONDS_BETWEEN_RETRIES = 2; private static final int DEFAULT_FS_BUFFER_SIZE = 1 << 18; // 256KB + private static final Pattern SNAPSHOT_JAR = Pattern.compile(".*\\-SNAPSHOT(-selfcontained)?\\.jar$"); public static Path distributedClassPath(String path) { @@ -90,9 +91,21 @@ public static Path distributedClassPath(Path base) return new Path(base, "classpath"); } + /** + * Uploads jar files to hdfs and configures the classpath. + * Snapshot jar files are uploaded to intermediateClasspath and not shared across multiple jobs. + * Non-Snapshot jar files are uploaded to a distributedClasspath and shared across multiple jobs. + * + * @param distributedClassPath classpath shared across multiple jobs + * @param intermediateClassPath classpath exclusive for this job. used to upload SNAPSHOT jar files. + * @param job job to run + * + * @throws IOException + */ public static void setupClasspath( - Path distributedClassPath, - Job job + final Path distributedClassPath, + final Path intermediateClassPath, + final Job job ) throws IOException { @@ -111,32 +124,158 @@ public static void setupClasspath( } for (String jarFilePath : jarFiles) { - File jarFile = new File(jarFilePath); + + final File jarFile = new File(jarFilePath); if (jarFile.getName().endsWith(".jar")) { - final Path hdfsPath = new Path(distributedClassPath, jarFile.getName()); - - if (!existing.contains(hdfsPath)) { - if (jarFile.getName().matches(".*SNAPSHOT(-selfcontained)?\\.jar$") || !fs.exists(hdfsPath)) { - log.info("Uploading jar to path[%s]", hdfsPath); - ByteStreams.copy( - Files.newInputStreamSupplier(jarFile), - new OutputSupplier() + try { + RetryUtils.retry( + new Callable() + { + @Override + public Boolean call() throws Exception { - @Override - public OutputStream getOutput() throws IOException - { - return fs.create(hdfsPath); + if (isSnapshot(jarFile)) { + addSnapshotJarToClassPath(jarFile, intermediateClassPath, fs, job); + } else { + addJarToClassPath(jarFile, distributedClassPath, intermediateClassPath, fs, job); } + return true; } - ); - } + }, + shouldRetryPredicate(), + NUM_RETRIES + ); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + } + } - existing.add(hdfsPath); + public static final Predicate shouldRetryPredicate() + { + return new Predicate() + { + @Override + public boolean apply(Throwable input) + { + if (input == null) { + return false; + } + if (input instanceof IOException) { + return true; } + return apply(input.getCause()); + } + }; + } - DistributedCache.addFileToClassPath(hdfsPath, conf, fs); + static void addJarToClassPath( + File jarFile, + Path distributedClassPath, + Path intermediateClassPath, + FileSystem fs, + Job job + ) + throws IOException + { + // Create distributed directory if it does not exist. + // rename will always fail if destination does not exist. + fs.mkdirs(distributedClassPath); + + // Non-snapshot jar files are uploaded to the shared classpath. + final Path hdfsPath = new Path(distributedClassPath, jarFile.getName()); + if (!fs.exists(hdfsPath)) { + // Muliple jobs can try to upload the jar here, + // to avoid them from overwriting files, first upload to intermediateClassPath and then rename to the distributedClasspath. + final Path intermediateHdfsPath = new Path(intermediateClassPath, jarFile.getName()); + uploadJar(jarFile, intermediateHdfsPath, fs); + IOException exception = null; + try { + log.info("Renaming jar to path[%s]", hdfsPath); + fs.rename(intermediateHdfsPath, hdfsPath); + if (!fs.exists(hdfsPath)) { + throw new IOException( + String.format( + "File does not exist even after moving from[%s] to [%s]", + intermediateHdfsPath, + hdfsPath + ) + ); + } + } + catch (IOException e) { + // rename failed, possibly due to race condition. check if some other job has uploaded the jar file. + try { + if (!fs.exists(hdfsPath)) { + log.error(e, "IOException while Renaming jar file"); + exception = e; + } + } + catch (IOException e1) { + e.addSuppressed(e1); + exception = e; + } + } + finally { + try { + if (fs.exists(intermediateHdfsPath)) { + fs.delete(intermediateHdfsPath, false); + } + } + catch (IOException e) { + if (exception == null) { + exception = e; + } else { + exception.addSuppressed(e); + } + } + if (exception != null) { + throw exception; + } } } + job.addFileToClassPath(hdfsPath); + } + + static void addSnapshotJarToClassPath( + File jarFile, + Path intermediateClassPath, + FileSystem fs, + Job job + ) throws IOException + { + // Snapshot jars are uploaded to non shared intermediate directory. + final Path hdfsPath = new Path(intermediateClassPath, jarFile.getName()); + + // existing is used to prevent uploading file multiple times in same run. + if (!existing.contains(hdfsPath)) { + uploadJar(jarFile, hdfsPath, fs); + existing.add(hdfsPath); + } + job.addFileToClassPath(hdfsPath); + } + + static void uploadJar(File jarFile, final Path path, final FileSystem fs) throws IOException + { + log.info("Uploading jar to path[%s]", path); + ByteStreams.copy( + Files.newInputStreamSupplier(jarFile), + new OutputSupplier() + { + @Override + public OutputStream getOutput() throws IOException + { + return fs.create(path); + } + } + ); + } + + static boolean isSnapshot(File jarFile) + { + return SNAPSHOT_JAR.matcher(jarFile.getName()).matches(); } public static void injectSystemProperties(Job job) diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/updater/HadoopConverterJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/updater/HadoopConverterJob.java index a1a411bf3ffb..0e13ed00cf59 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/updater/HadoopConverterJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/updater/HadoopConverterJob.java @@ -140,11 +140,17 @@ public static Path getTaskPath(JobID jobID, TaskAttemptID taskAttemptID, Path wo return new Path(getJobPath(jobID, workingDirectory), taskAttemptID.toString()); } + public static Path getJobClassPathDir(String jobName, Path workingDirectory) throws IOException + { + return new Path(workingDirectory, jobName.replace(":", "")); + } + public static void cleanup(Job job) throws IOException { final Path jobDir = getJobPath(job.getJobID(), job.getWorkingDirectory()); final FileSystem fs = jobDir.getFileSystem(job.getConfiguration()); fs.delete(jobDir, true); + fs.delete(getJobClassPathDir(job.getJobName(), job.getWorkingDirectory()), true); } @@ -231,7 +237,11 @@ public List run() throws IOException job.setMapSpeculativeExecution(false); job.setOutputFormatClass(ConvertingOutputFormat.class); - JobHelper.setupClasspath(JobHelper.distributedClassPath(jobConf.getWorkingDirectory()), job); + JobHelper.setupClasspath( + JobHelper.distributedClassPath(jobConf.getWorkingDirectory()), + JobHelper.distributedClassPath(getJobClassPathDir(job.getJobName(), jobConf.getWorkingDirectory())), + job + ); Throwable throwable = null; try { diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/HdfsClasspathSetupTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/HdfsClasspathSetupTest.java new file mode 100644 index 000000000000..6204c04d1de1 --- /dev/null +++ b/indexing-hadoop/src/test/java/io/druid/indexer/HdfsClasspathSetupTest.java @@ -0,0 +1,203 @@ +/* +* Licensed to Metamarkets Group Inc. (Metamarkets) under one +* or more contributor license agreements. See the NOTICE file +* distributed with this work for additional information +* regarding copyright ownership. Metamarkets licenses this file +* to you under the Apache License, Version 2.0 (the +* "License"); you may not use this file except in compliance +* with the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, +* software distributed under the License is distributed on an +* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +* KIND, either express or implied. See the License for the +* specific language governing permissions and limitations +* under the License. +*/ + +package io.druid.indexer; + +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; +import com.metamx.common.StringUtils; +import io.druid.common.utils.UUIDUtils; +import java.io.ByteArrayInputStream; +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.StandardCopyOption; +import java.util.ArrayList; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.Callable; +import java.util.concurrent.CyclicBarrier; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import junit.framework.Assert; +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.MRJobConfig; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class HdfsClasspathSetupTest +{ + private static MiniDFSCluster miniCluster; + private static File hdfsTmpDir; + private static Configuration conf; + private static String dummyJarString = "This is a test jar file."; + private File dummyJarFile; + private Path finalClasspath; + private Path intermediatePath; + @Rule + public final TemporaryFolder tempFolder = new TemporaryFolder(); + + @BeforeClass + public static void setupStatic() throws IOException, ClassNotFoundException + { + hdfsTmpDir = File.createTempFile("hdfsClasspathSetupTest", "dir"); + hdfsTmpDir.deleteOnExit(); + if (!hdfsTmpDir.delete()) { + throw new IOException(String.format("Unable to delete hdfsTmpDir [%s]", hdfsTmpDir.getAbsolutePath())); + } + conf = new Configuration(true); + conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, hdfsTmpDir.getAbsolutePath()); + miniCluster = new MiniDFSCluster.Builder(conf).build(); + } + + @Before + public void setUp() throws IOException + { + // intermedatePath and finalClasspath are relative to hdfsTmpDir directory. + intermediatePath = new Path(String.format("/tmp/classpath/%s", UUIDUtils.generateUuid())); + finalClasspath = new Path(String.format("/tmp/intermediate/%s", UUIDUtils.generateUuid())); + dummyJarFile = tempFolder.newFile("dummy-test.jar"); + Files.copy( + new ByteArrayInputStream(StringUtils.toUtf8(dummyJarString)), + dummyJarFile.toPath(), + StandardCopyOption.REPLACE_EXISTING + ); + } + + @AfterClass + public static void tearDownStatic() throws IOException + { + if (miniCluster != null) { + miniCluster.shutdown(true); + } + } + + @After + public void tearDown() throws IOException + { + dummyJarFile.delete(); + Assert.assertFalse(dummyJarFile.exists()); + miniCluster.getFileSystem().delete(finalClasspath, true); + Assert.assertFalse(miniCluster.getFileSystem().exists(finalClasspath)); + miniCluster.getFileSystem().delete(intermediatePath, true); + Assert.assertFalse(miniCluster.getFileSystem().exists(intermediatePath)); + } + + @Test + public void testAddSnapshotJarToClasspath() throws IOException + { + Job job = Job.getInstance(conf, "test-job"); + DistributedFileSystem fs = miniCluster.getFileSystem(); + Path intermediatePath = new Path("/tmp/classpath"); + JobHelper.addSnapshotJarToClassPath(dummyJarFile, intermediatePath, fs, job); + Path expectedJarPath = new Path(intermediatePath, dummyJarFile.getName()); + // check file gets uploaded to HDFS + Assert.assertTrue(fs.exists(expectedJarPath)); + // check file gets added to the classpath + Assert.assertEquals(expectedJarPath.toString(), job.getConfiguration().get(MRJobConfig.CLASSPATH_FILES)); + Assert.assertEquals(dummyJarString, StringUtils.fromUtf8(IOUtils.toByteArray(fs.open(expectedJarPath)))); + } + + @Test + public void testAddNonSnapshotJarToClasspath() throws IOException + { + Job job = Job.getInstance(conf, "test-job"); + DistributedFileSystem fs = miniCluster.getFileSystem(); + JobHelper.addJarToClassPath(dummyJarFile, finalClasspath, intermediatePath, fs, job); + Path expectedJarPath = new Path(finalClasspath, dummyJarFile.getName()); + // check file gets uploaded to final HDFS path + Assert.assertTrue(fs.exists(expectedJarPath)); + // check that the intermediate file gets deleted + Assert.assertFalse(fs.exists(new Path(intermediatePath, dummyJarFile.getName()))); + // check file gets added to the classpath + Assert.assertEquals(expectedJarPath.toString(), job.getConfiguration().get(MRJobConfig.CLASSPATH_FILES)); + Assert.assertEquals(dummyJarString, StringUtils.fromUtf8(IOUtils.toByteArray(fs.open(expectedJarPath)))); + } + + @Test + public void testIsSnapshot() + { + Assert.assertTrue(JobHelper.isSnapshot(new File("test-SNAPSHOT.jar"))); + Assert.assertTrue(JobHelper.isSnapshot(new File("test-SNAPSHOT-selfcontained.jar"))); + Assert.assertFalse(JobHelper.isSnapshot(new File("test.jar"))); + Assert.assertFalse(JobHelper.isSnapshot(new File("test-selfcontained.jar"))); + Assert.assertFalse(JobHelper.isSnapshot(new File("iAmNotSNAPSHOT.jar"))); + Assert.assertFalse(JobHelper.isSnapshot(new File("iAmNotSNAPSHOT-selfcontained.jar"))); + + } + + @Test + public void testConcurrentUpload() throws IOException, InterruptedException, ExecutionException, TimeoutException + { + final int concurrency = 10; + ListeningExecutorService pool = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(concurrency)); + // barrier ensures that all jobs try to add files to classpath at same time. + final CyclicBarrier barrier = new CyclicBarrier(concurrency); + final DistributedFileSystem fs = miniCluster.getFileSystem(); + final Path expectedJarPath = new Path(finalClasspath, dummyJarFile.getName()); + List> futures = new ArrayList<>(); + + for (int i = 0; i < concurrency; i++) { + futures.add( + pool.submit( + new Callable() + { + @Override + public Boolean call() throws Exception + { + int id = barrier.await(); + Job job = Job.getInstance(conf, "test-job-" + id); + Path intermediatePathForJob = new Path(intermediatePath, "job-" + id); + JobHelper.addJarToClassPath(dummyJarFile, finalClasspath, intermediatePathForJob, fs, job); + // check file gets uploaded to final HDFS path + Assert.assertTrue(fs.exists(expectedJarPath)); + // check that the intermediate file is not present + Assert.assertFalse(fs.exists(new Path(intermediatePathForJob, dummyJarFile.getName()))); + // check file gets added to the classpath + Assert.assertEquals( + expectedJarPath.toString(), + job.getConfiguration().get(MRJobConfig.CLASSPATH_FILES) + ); + return true; + } + } + ) + ); + } + + Futures.allAsList(futures).get(30, TimeUnit.SECONDS); + + pool.shutdownNow(); + } + +} diff --git a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java index dcc8e56db47a..fedec3a700df 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java @@ -26,6 +26,8 @@ import com.metamx.emitter.service.ServiceEmitter; import com.metamx.metrics.MonitorScheduler; import io.druid.client.FilteredServerView; +import io.druid.client.cache.Cache; +import io.druid.client.cache.CacheConfig; import io.druid.indexing.common.actions.SegmentInsertAction; import io.druid.indexing.common.actions.TaskActionClient; import io.druid.indexing.common.actions.TaskActionClientFactory; @@ -70,6 +72,8 @@ public class TaskToolbox private final SegmentLoader segmentLoader; private final ObjectMapper objectMapper; private final File taskWorkDir; + private final Cache cache; + private final CacheConfig cacheConfig; public TaskToolbox( TaskConfig config, @@ -87,7 +91,9 @@ public TaskToolbox( MonitorScheduler monitorScheduler, SegmentLoader segmentLoader, ObjectMapper objectMapper, - final File taskWorkDir + final File taskWorkDir, + Cache cache, + CacheConfig cacheConfig ) { this.config = config; @@ -106,6 +112,8 @@ public TaskToolbox( this.segmentLoader = segmentLoader; this.objectMapper = objectMapper; this.taskWorkDir = taskWorkDir; + this.cache = cache; + this.cacheConfig = cacheConfig; } public TaskConfig getConfig() @@ -208,4 +216,14 @@ public File getTaskWorkDir() { return taskWorkDir; } + + public Cache getCache() + { + return cache; + } + + public CacheConfig getCacheConfig() + { + return cacheConfig; + } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java index 404fe42cf84b..16628ef345a5 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java @@ -22,6 +22,8 @@ import com.metamx.emitter.service.ServiceEmitter; import com.metamx.metrics.MonitorScheduler; import io.druid.client.FilteredServerView; +import io.druid.client.cache.Cache; +import io.druid.client.cache.CacheConfig; import io.druid.guice.annotations.Processing; import io.druid.indexing.common.actions.TaskActionClientFactory; import io.druid.indexing.common.config.TaskConfig; @@ -55,6 +57,8 @@ public class TaskToolboxFactory private final MonitorScheduler monitorScheduler; private final SegmentLoaderFactory segmentLoaderFactory; private final ObjectMapper objectMapper; + private final Cache cache; + private final CacheConfig cacheConfig; @Inject public TaskToolboxFactory( @@ -71,7 +75,9 @@ public TaskToolboxFactory( @Processing ExecutorService queryExecutorService, MonitorScheduler monitorScheduler, SegmentLoaderFactory segmentLoaderFactory, - ObjectMapper objectMapper + ObjectMapper objectMapper, + Cache cache, + CacheConfig cacheConfig ) { this.config = config; @@ -88,11 +94,13 @@ public TaskToolboxFactory( this.monitorScheduler = monitorScheduler; this.segmentLoaderFactory = segmentLoaderFactory; this.objectMapper = objectMapper; + this.cache = cache; + this.cacheConfig = cacheConfig; } public TaskToolbox build(Task task) { - final File taskWorkDir = new File(new File(config.getBaseTaskDir(), task.getId()), "work"); + final File taskWorkDir = config.getTaskWorkDir(task.getId()); return new TaskToolbox( config, @@ -110,7 +118,9 @@ public TaskToolbox build(Task task) monitorScheduler, segmentLoaderFactory.manufacturate(taskWorkDir), objectMapper, - taskWorkDir + taskWorkDir, + cache, + cacheConfig ); } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/RemoteTaskActionClient.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/RemoteTaskActionClient.java index b048bf25346f..4d736023b1b6 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/RemoteTaskActionClient.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/RemoteTaskActionClient.java @@ -108,7 +108,7 @@ public RetType submit(TaskAction taskAction) throws IOExcepti throw Throwables.propagate(e); } - if (response.getStatus().getCode() / 200 == 1) { + if (response.getStatus().getCode() / 100 == 2) { final Map responseDict = jsonMapper.readValue( response.getContent(), new TypeReference>() diff --git a/indexing-service/src/main/java/io/druid/indexing/common/config/TaskConfig.java b/indexing-service/src/main/java/io/druid/indexing/common/config/TaskConfig.java index 4e17326d41f2..da4a03f329c0 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/config/TaskConfig.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/config/TaskConfig.java @@ -20,6 +20,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ImmutableList; +import org.joda.time.Period; import java.io.File; import java.util.List; @@ -30,6 +31,10 @@ public class TaskConfig "org.apache.hadoop:hadoop-client:2.3.0" ); + private static final Period DEFAULT_DIRECTORY_LOCK_TIMEOUT = new Period("PT10M"); + + private static final Period DEFAULT_GRACEFUL_SHUTDOWN_TIMEOUT = new Period("PT5M"); + @JsonProperty private final String baseDir; @@ -45,13 +50,21 @@ public class TaskConfig @JsonProperty private final List defaultHadoopCoordinates; + @JsonProperty + private final Period gracefulShutdownTimeout; + + @JsonProperty + private final Period directoryLockTimeout; + @JsonCreator public TaskConfig( @JsonProperty("baseDir") String baseDir, @JsonProperty("baseTaskDir") String baseTaskDir, @JsonProperty("hadoopWorkingPath") String hadoopWorkingPath, @JsonProperty("defaultRowFlushBoundary") Integer defaultRowFlushBoundary, - @JsonProperty("defaultHadoopCoordinates") List defaultHadoopCoordinates + @JsonProperty("defaultHadoopCoordinates") List defaultHadoopCoordinates, + @JsonProperty("gracefulShutdownTimeout") Period gracefulShutdownTimeout, + @JsonProperty("directoryLockTimeout") Period directoryLockTimeout ) { this.baseDir = baseDir == null ? "/tmp" : baseDir; @@ -61,6 +74,12 @@ public TaskConfig( this.defaultHadoopCoordinates = defaultHadoopCoordinates == null ? DEFAULT_DEFAULT_HADOOP_COORDINATES : defaultHadoopCoordinates; + this.gracefulShutdownTimeout = gracefulShutdownTimeout == null + ? DEFAULT_GRACEFUL_SHUTDOWN_TIMEOUT + : gracefulShutdownTimeout; + this.directoryLockTimeout = directoryLockTimeout == null + ? DEFAULT_DIRECTORY_LOCK_TIMEOUT + : directoryLockTimeout; } @JsonProperty @@ -75,6 +94,21 @@ public File getBaseTaskDir() return baseTaskDir; } + public File getTaskDir(String taskId) + { + return new File(baseTaskDir, taskId); + } + + public File getTaskWorkDir(String taskId) + { + return new File(getTaskDir(taskId), "work"); + } + + public File getTaskLockFile(String taskId) + { + return new File(getTaskDir(taskId), "lock"); + } + @JsonProperty public String getHadoopWorkingPath() { @@ -93,6 +127,18 @@ public List getDefaultHadoopCoordinates() return defaultHadoopCoordinates; } + @JsonProperty + public Period getGracefulShutdownTimeout() + { + return gracefulShutdownTimeout; + } + + @JsonProperty + public Period getDirectoryLockTimeout() + { + return directoryLockTimeout; + } + private String defaultDir(String configParameter, final String defaultVal) { if (configParameter == null) { diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/AbstractTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/AbstractTask.java index 3d3df6b363b6..4bbacf3c9f48 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/AbstractTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/AbstractTask.java @@ -128,6 +128,19 @@ public String getClasspathPrefix() return null; } + @Override + public boolean canRestore() + { + return false; + } + + @Override + public void stopGracefully() + { + // Should not be called when canRestore = false. + throw new UnsupportedOperationException("Cannot stop gracefully"); + } + @Override public String toString() { diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java index b8a2c7e4190a..280acd34ed55 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java @@ -29,6 +29,7 @@ import com.metamx.emitter.EmittingLogger; import io.druid.data.input.Committer; import io.druid.data.input.Firehose; +import io.druid.data.input.FirehoseFactory; import io.druid.data.input.InputRow; import io.druid.indexing.common.TaskLock; import io.druid.indexing.common.TaskStatus; @@ -48,6 +49,9 @@ import io.druid.segment.realtime.FireDepartment; import io.druid.segment.realtime.RealtimeMetricsMonitor; import io.druid.segment.realtime.SegmentPublisher; +import io.druid.segment.realtime.firehose.ClippedFirehoseFactory; +import io.druid.segment.realtime.firehose.EventReceiverFirehoseFactory; +import io.druid.segment.realtime.firehose.TimedShutoffFirehoseFactory; import io.druid.segment.realtime.plumber.Committers; import io.druid.segment.realtime.plumber.Plumber; import io.druid.segment.realtime.plumber.PlumberSchool; @@ -62,6 +66,7 @@ import java.io.IOException; import java.util.Map; import java.util.Random; +import java.util.concurrent.CountDownLatch; public class RealtimeIndexTask extends AbstractTask { @@ -104,6 +109,12 @@ private static String makeDatasource(FireDepartment fireDepartment) @JsonIgnore private volatile Plumber plumber = null; + @JsonIgnore + private volatile Firehose firehose = null; + + @JsonIgnore + private volatile boolean stopped = false; + @JsonIgnore private volatile QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate = null; @@ -275,13 +286,14 @@ public String getVersion(final Interval interval) lockingSegmentAnnouncer, segmentPublisher, toolbox.getNewSegmentServerView(), - toolbox.getQueryExecutorService() + toolbox.getQueryExecutorService(), + toolbox.getCache(), + toolbox.getCacheConfig(), + toolbox.getObjectMapper() ); this.plumber = plumberSchool.findPlumber(dataSchema, tuningConfig, fireDepartment.getMetrics()); - // Delay firehose connection to avoid claiming input resources while the plumber is starting up. - Firehose firehose = null; Supplier committerSupplier = null; try { @@ -290,12 +302,14 @@ public String getVersion(final Interval interval) // Set up metrics emission toolbox.getMonitorScheduler().addMonitor(metricsMonitor); - // Set up firehose - firehose = spec.getIOConfig().getFirehoseFactory().connect(spec.getDataSchema().getParser()); + // Delay firehose connection to avoid claiming input resources while the plumber is starting up. + final FirehoseFactory firehoseFactory = spec.getIOConfig().getFirehoseFactory(); + final boolean firehoseDrainableByClosing = isFirehoseDrainableByClosing(firehoseFactory); + firehose = firehoseFactory.connect(spec.getDataSchema().getParser()); committerSupplier = Committers.supplierFromFirehose(firehose); // Time to read data! - while (firehose.hasMore()) { + while ((!stopped || firehoseDrainableByClosing) && firehose.hasMore()) { final InputRow inputRow; try { @@ -332,8 +346,38 @@ public String getVersion(final Interval interval) finally { if (normalExit) { try { - plumber.persist(committerSupplier.get()); - plumber.finishJob(); + if (!stopped) { + // Hand off all pending data + log.info("Persisting and handing off pending data."); + plumber.persist(committerSupplier.get()); + plumber.finishJob(); + } else { + log.info("Persisting pending data without handoff, in preparation for restart."); + final Committer committer = committerSupplier.get(); + final CountDownLatch persistLatch = new CountDownLatch(1); + plumber.persist( + new Committer() + { + @Override + public Object getMetadata() + { + return committer.getMetadata(); + } + + @Override + public void run() + { + try { + committer.run(); + } + finally { + persistLatch.countDown(); + } + } + } + ); + persistLatch.await(); + } } catch (Exception e) { log.makeAlert(e, "Failed to finish realtime task").emit(); @@ -347,15 +391,67 @@ public String getVersion(final Interval interval) } } + log.info("Job done!"); return TaskStatus.success(getId()); } + @Override + public boolean canRestore() + { + return true; + } + + @Override + public void stopGracefully() + { + try { + synchronized (this) { + if (!stopped) { + stopped = true; + log.info("Gracefully stopping."); + if (isFirehoseDrainableByClosing(spec.getIOConfig().getFirehoseFactory())) { + firehose.close(); + } else { + log.debug("Cannot drain firehose[%s] by closing, so skipping closing.", firehose); + } + } + } + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + + /** + * Public for tests. + */ + @JsonIgnore + public Firehose getFirehose() + { + return firehose; + } + @JsonProperty("spec") public FireDepartment getRealtimeIngestionSchema() { return spec; } + /** + * Is a firehose from this factory drainable by closing it? If so, we should drain on stopGracefully rather than + * abruptly stopping. + *

+ * This is a hack to get around the fact that the Firehose and FirehoseFactory interfaces do not help us do this. + */ + private static boolean isFirehoseDrainableByClosing(FirehoseFactory firehoseFactory) + { + return firehoseFactory instanceof EventReceiverFirehoseFactory + || (firehoseFactory instanceof TimedShutoffFirehoseFactory + && isFirehoseDrainableByClosing(((TimedShutoffFirehoseFactory) firehoseFactory).getDelegateFactory())) + || (firehoseFactory instanceof ClippedFirehoseFactory + && isFirehoseDrainableByClosing(((ClippedFirehoseFactory) firehoseFactory).getDelegate())); + } + public static class TaskActionSegmentPublisher implements SegmentPublisher { final Task task; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java b/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java index 2f3e472b76f2..f03937e74dea 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java @@ -61,6 +61,7 @@ public interface Task { /** * Returns ID of this task. Must be unique across all tasks ever created. + * * @return task ID */ public String getId(); @@ -68,6 +69,7 @@ public interface Task /** * Returns group ID of this task. Tasks with the same group ID can share locks. If tasks do not need to share locks, * a common convention is to set group ID equal to task ID. + * * @return task group ID */ public String getGroupId(); @@ -75,12 +77,14 @@ public interface Task /** * Returns a {@link io.druid.indexing.common.task.TaskResource} for this task. Task resources define specific * worker requirements a task may require. + * * @return {@link io.druid.indexing.common.task.TaskResource} for this task */ public TaskResource getTaskResource(); /** * Returns a descriptive label for this task type. Used for metrics emission and logging. + * * @return task type label */ public String getType(); @@ -89,7 +93,7 @@ public interface Task * Get the nodeType for if/when this task publishes on zookeeper. * * @return the nodeType to use when publishing the server to zookeeper. null if the task doesn't expect to - * publish to zookeeper. + * publish to zookeeper. */ public String getNodeType(); @@ -101,7 +105,9 @@ public interface Task /** * Returns query runners for this task. If this task is not meant to answer queries over its datasource, this method * should return null. + * * @param query result type + * * @return query runners for this task */ public QueryRunner getQueryRunner(Query query); @@ -116,7 +122,7 @@ public interface Task * Execute preflight actions for a task. This can be used to acquire locks, check preconditions, and so on. The * actions must be idempotent, since this method may be executed multiple times. This typically runs on the * coordinator. If this method throws an exception, the task should be considered a failure. - * + *

* This method must be idempotent, as it may be run multiple times per task. * * @param taskActionClient action client for this task (not the full toolbox) @@ -127,6 +133,20 @@ public interface Task */ public boolean isReady(TaskActionClient taskActionClient) throws Exception; + /** + * Returns whether or not this task can restore its progress from its on-disk working directory. Restorable tasks + * may be started with a non-empty working directory. Tasks that exit uncleanly may still have a chance to attempt + * restores, meaning that restorable tasks should be able to deal with potentially partially written on-disk state. + */ + public boolean canRestore(); + + /** + * Asks a task to arrange for its "run" method to exit promptly. This method will only be called if + * {@link #canRestore()} returns true. Tasks that take too long to stop gracefully will be terminated with + * extreme prejudice. + */ + public void stopGracefully(); + /** * Execute a task. This typically runs on a worker as determined by a TaskRunner, and will be run while * holding a lock on our dataSource and implicit lock interval (if any). If this method throws an exception, the task diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunner.java index cf7279f87d0d..cbd3e6632025 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunner.java @@ -17,6 +17,8 @@ package io.druid.indexing.overlord; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.CharMatcher; import com.google.common.base.Joiner; @@ -28,17 +30,21 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import com.google.common.io.ByteSink; import com.google.common.io.ByteSource; import com.google.common.io.ByteStreams; import com.google.common.io.Closer; +import com.google.common.io.FileWriteMode; import com.google.common.io.Files; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; import com.google.inject.Inject; import com.metamx.common.ISE; +import com.metamx.common.Pair; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.emitter.EmittingLogger; +import io.druid.concurrent.Execs; import io.druid.guice.annotations.Self; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.config.TaskConfig; @@ -50,6 +56,8 @@ import io.druid.tasklogs.TaskLogPusher; import io.druid.tasklogs.TaskLogStreamer; import org.apache.commons.io.FileUtils; +import org.joda.time.DateTime; +import org.joda.time.Interval; import java.io.File; import java.io.IOException; @@ -62,7 +70,7 @@ import java.util.Properties; import java.util.UUID; import java.util.concurrent.Callable; -import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; /** * Runs tasks in separate processes using the "internal peon" verb. @@ -71,6 +79,7 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer { private static final EmittingLogger log = new EmittingLogger(ForkingTaskRunner.class); private static final String CHILD_PROPERTY_PREFIX = "druid.indexer.fork.property."; + private static final String TASK_RESTORE_FILENAME = "restore.json"; private final ForkingTaskRunnerConfig config; private final TaskConfig taskConfig; private final Properties props; @@ -82,6 +91,8 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer private final Map tasks = Maps.newHashMap(); + private volatile boolean stopping = false; + @Inject public ForkingTaskRunner( ForkingTaskRunnerConfig config, @@ -101,7 +112,51 @@ public ForkingTaskRunner( this.node = node; this.portFinder = new PortFinder(config.getStartPort()); - this.exec = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(workerConfig.getCapacity())); + this.exec = MoreExecutors.listeningDecorator( + Execs.multiThreaded(workerConfig.getCapacity(), "forking-task-runner-%d") + ); + } + + @Override + public List>> restore() + { + final File restoreFile = getRestoreFile(); + final TaskRestoreInfo taskRestoreInfo; + if (restoreFile.exists()) { + try { + taskRestoreInfo = jsonMapper.readValue(restoreFile, TaskRestoreInfo.class); + } + catch (Exception e) { + log.error(e, "Failed to read restorable tasks from file[%s]. Skipping restore.", restoreFile); + return ImmutableList.of(); + } + } else { + return ImmutableList.of(); + } + + final List>> retVal = Lists.newArrayList(); + for (final String taskId : taskRestoreInfo.getRunningTasks()) { + try { + final File taskFile = new File(taskConfig.getTaskDir(taskId), "task.json"); + final Task task = jsonMapper.readValue(taskFile, Task.class); + + if (!task.getId().equals(taskId)) { + throw new ISE("WTF?! Task[%s] restore file had wrong id[%s].", taskId, task.getId()); + } + + if (task.canRestore()) { + log.info("Restoring task[%s].", task.getId()); + retVal.add(Pair.of(task, run(task))); + } + } + catch (Exception e) { + log.warn(e, "Failed to restore task[%s]. Trying to restore other tasks.", taskId); + } + } + + log.info("Restored %,d tasks.", retVal.size()); + + return retVal; } @Override @@ -112,7 +167,7 @@ public ListenableFuture run(final Task task) tasks.put( task.getId(), new ForkingTaskRunnerWorkItem( - task.getId(), + task, exec.submit( new Callable() { @@ -120,11 +175,22 @@ public ListenableFuture run(final Task task) public TaskStatus call() { final String attemptUUID = UUID.randomUUID().toString(); - final File taskDir = new File(taskConfig.getBaseTaskDir(), task.getId()); + final File taskDir = taskConfig.getTaskDir(task.getId()); final File attemptDir = new File(taskDir, attemptUUID); final ProcessHolder processHolder; - final int childPort = portFinder.findUnusedPort(); + final int childPort; + final int childChatHandlerPort; + + if (config.isSeparateIngestionEndpoint()) { + Pair portPair = portFinder.findTwoConsecutiveUnusedPorts(); + childPort = portPair.lhs; + childChatHandlerPort = portPair.rhs; + } else { + childPort = portFinder.findUnusedPort(); + childChatHandlerPort = -1; + } + try { final Closer closer = Closer.create(); try { @@ -132,9 +198,9 @@ public TaskStatus call() throw new IOException(String.format("Could not create directories: %s", attemptDir)); } - final File taskFile = new File(attemptDir, "task.json"); + final File taskFile = new File(taskDir, "task.json"); final File statusFile = new File(attemptDir, "status.json"); - final File logFile = new File(attemptDir, "log"); + final File logFile = new File(taskDir, "log"); // time to adjust process holders synchronized (tasks) { @@ -231,6 +297,14 @@ public TaskStatus call() command.add(String.format("-Ddruid.host=%s", childHost)); command.add(String.format("-Ddruid.port=%d", childPort)); + if(config.isSeparateIngestionEndpoint()) { + command.add(String.format("-Ddruid.indexer.task.chathandler.service=%s", "placeholder/serviceName")); + // Actual serviceName will be passed by the EventReceiverFirehose when it registers itself with ChatHandlerProvider + // Thus, "placeholder/serviceName" will be ignored + command.add(String.format("-Ddruid.indexer.task.chathandler.host=%s", childHost)); + command.add(String.format("-Ddruid.indexer.task.chathandler.port=%d", childChatHandlerPort)); + } + command.add("io.druid.cli.Main"); command.add("internal"); command.add("peon"); @@ -242,7 +316,9 @@ public TaskStatus call() command.add(nodeType); } - jsonMapper.writeValue(taskFile, task); + if (!taskFile.exists()) { + jsonMapper.writeValue(taskFile, task); + } log.info("Running command: %s", Joiner.on(" ").join(command)); taskWorkItem.processHolder = new ProcessHolder( @@ -258,7 +334,8 @@ public TaskStatus call() log.info("Logging task %s output to: %s", task.getId(), logFile); boolean runFailed = true; - try (final OutputStream toLogfile = Files.asByteSink(logFile).openBufferedStream()) { + final ByteSink logSink = Files.asByteSink(logFile, FileWriteMode.APPEND); + try (final OutputStream toLogfile = logSink.openStream()) { ByteStreams.copy(processHolder.process.getInputStream(), toLogfile); final int statusCode = processHolder.process.waitFor(); log.info("Process exited with status[%d] for task: %s", statusCode, task.getId()); @@ -297,10 +374,26 @@ public TaskStatus call() if (taskWorkItem != null && taskWorkItem.processHolder != null) { taskWorkItem.processHolder.process.destroy(); } + if (!stopping) { + saveRunningTasks(); + } + } + if(childChatHandlerPort > 0) { + portFinder.markPortUnused(childChatHandlerPort); + } + + try { + if (!stopping && taskDir.exists()) { + log.info("Removing task directory: %s", taskDir); + FileUtils.deleteDirectory(taskDir); + } + } + catch (Exception e) { + log.makeAlert(e, "Failed to delete task directory") + .addData("taskDir", taskDir.toString()) + .addData("task", task.getId()) + .emit(); } - portFinder.markPortUnused(childPort); - log.info("Removing temporary directory: %s", attemptDir); - FileUtils.deleteDirectory(attemptDir); } catch (Exception e) { log.error(e, "Suppressing exception caught while cleaning up task"); @@ -312,7 +405,7 @@ public TaskStatus call() ) ); } - + saveRunningTasks(); return tasks.get(task.getId()).getResult(); } } @@ -320,16 +413,41 @@ public TaskStatus call() @LifecycleStop public void stop() { - synchronized (tasks) { - exec.shutdown(); + stopping = true; + exec.shutdown(); + synchronized (tasks) { for (ForkingTaskRunnerWorkItem taskWorkItem : tasks.values()) { if (taskWorkItem.processHolder != null) { - log.info("Destroying process: %s", taskWorkItem.processHolder.process); - taskWorkItem.processHolder.process.destroy(); + log.info("Closing output stream to task[%s].", taskWorkItem.getTask().getId()); + try { + taskWorkItem.processHolder.process.getOutputStream().close(); + } + catch (Exception e) { + log.warn(e, "Failed to close stdout to task[%s]. Destroying task.", taskWorkItem.getTask().getId()); + taskWorkItem.processHolder.process.destroy(); + } } } } + + final DateTime start = new DateTime(); + final long timeout = new Interval(start, taskConfig.getGracefulShutdownTimeout()).toDurationMillis(); + + // Things should be terminating now. Wait for it to happen so logs can be uploaded and all that good stuff. + log.info("Waiting %,dms for shutdown.", timeout); + if (timeout > 0) { + try { + exec.awaitTermination(timeout, TimeUnit.MILLISECONDS); + log.info("Finished stopping in %,dms.", System.currentTimeMillis() - start.getMillis()); + } + catch (InterruptedException e) { + log.warn(e, "Interrupted while waiting for executor to finish."); + Thread.currentThread().interrupt(); + } + } else { + log.warn("Ran out of time, not waiting for executor to finish!"); + } } @Override @@ -423,17 +541,68 @@ public InputStream openStream() throws IOException ); } + // Save running tasks to a file, so they can potentially be restored on next startup. Suppresses exceptions that + // occur while saving. + private void saveRunningTasks() + { + final File restoreFile = getRestoreFile(); + final List theTasks = Lists.newArrayList(); + for (ForkingTaskRunnerWorkItem forkingTaskRunnerWorkItem : tasks.values()) { + theTasks.add(forkingTaskRunnerWorkItem.getTaskId()); + } + + try { + Files.createParentDirs(restoreFile); + jsonMapper.writeValue(restoreFile, new TaskRestoreInfo(theTasks)); + } + catch (Exception e) { + log.warn(e, "Failed to save tasks to restore file[%s]. Skipping this save.", restoreFile); + } + } + + private File getRestoreFile() + { + return new File(taskConfig.getBaseTaskDir(), TASK_RESTORE_FILENAME); + } + + private static class TaskRestoreInfo + { + @JsonProperty + private final List runningTasks; + + @JsonCreator + public TaskRestoreInfo( + @JsonProperty("runningTasks") List runningTasks + ) + { + this.runningTasks = runningTasks; + } + + public List getRunningTasks() + { + return runningTasks; + } + } + private static class ForkingTaskRunnerWorkItem extends TaskRunnerWorkItem { + private final Task task; + private volatile boolean shutdown = false; private volatile ProcessHolder processHolder = null; private ForkingTaskRunnerWorkItem( - String taskId, + Task task, ListenableFuture statusFuture ) { - super(taskId, statusFuture); + super(task.getId(), statusFuture); + this.task = task; + } + + public Task getTask() + { + return task; } } diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/PortFinder.java b/indexing-service/src/main/java/io/druid/indexing/overlord/PortFinder.java index be3c1d7412d4..3f0cb5954675 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/PortFinder.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/PortFinder.java @@ -19,6 +19,7 @@ import com.google.common.collect.Sets; import com.metamx.common.ISE; +import com.metamx.common.Pair; import java.io.IOException; import java.net.BindException; @@ -74,6 +75,17 @@ public synchronized int findUnusedPort() return port; } + public synchronized Pair findTwoConsecutiveUnusedPorts() + { + int firstPort = chooseNext(startPort); + while (!canBind(firstPort) || !canBind(firstPort + 1)) { + firstPort = chooseNext(firstPort + 1); + } + usedPorts.add(firstPort); + usedPorts.add(firstPort + 1); + return new Pair<>(firstPort, firstPort + 1); + } + public synchronized void markPortUnused(int port) { usedPorts.remove(port); diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java index 008729d89698..1a270e71e188 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java @@ -37,6 +37,7 @@ import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.SettableFuture; import com.metamx.common.ISE; +import com.metamx.common.Pair; import com.metamx.common.RE; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; @@ -296,6 +297,12 @@ public void stop() } } + @Override + public List>> restore() + { + return ImmutableList.of(); + } + @Override public Collection getWorkers() { diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskQueue.java b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskQueue.java index 682f72c63b6a..bfcbd382bfa2 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskQueue.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskQueue.java @@ -216,6 +216,9 @@ private void manage() throws InterruptedException log.info("Beginning management in %s.", config.getStartDelay()); Thread.sleep(config.getStartDelay().getMillis()); + // Ignore return value- we'll get the IDs and futures from getKnownTasks later. + taskRunner.restore(); + while (active) { giant.lock(); diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunner.java index ad5d2e2dd8b8..b5841d6836b1 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunner.java @@ -18,16 +18,24 @@ package io.druid.indexing.overlord; import com.google.common.util.concurrent.ListenableFuture; +import com.metamx.common.Pair; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.task.Task; import java.util.Collection; +import java.util.List; /** * Interface for handing off tasks. Managed by a {@link io.druid.indexing.overlord.TaskQueue}. */ public interface TaskRunner { + /** + * Some task runners can restart previously-running tasks after being bounced. This method does that, and returns + * the list of tasks (and status futures). + */ + public List>> restore(); + /** * Run a task. The returned status should be some kind of completed status. * @@ -45,6 +53,12 @@ public interface TaskRunner */ public void shutdown(String taskid); + /** + * Stop this task runner. This may block until currently-running tasks can be gracefully stopped. After calling + * stopping, "run" will not accept further tasks. + */ + public void stop(); + public Collection getRunningTasks(); public Collection getPendingTasks(); diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/ThreadPoolTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/overlord/ThreadPoolTaskRunner.java index 462179d0b156..a20d8d0f4d65 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/ThreadPoolTaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/ThreadPoolTaskRunner.java @@ -17,7 +17,6 @@ package io.druid.indexing.overlord; -import com.google.common.base.Function; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; @@ -29,54 +28,120 @@ import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; import com.google.inject.Inject; +import com.metamx.common.Pair; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.emitter.EmittingLogger; +import com.metamx.emitter.service.ServiceEmitter; +import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.concurrent.Execs; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.TaskToolboxFactory; +import io.druid.indexing.common.config.TaskConfig; import io.druid.indexing.common.task.Task; import io.druid.query.NoopQueryRunner; import io.druid.query.Query; import io.druid.query.QueryRunner; -import io.druid.query.QueryRunnerFactoryConglomerate; import io.druid.query.QuerySegmentWalker; import io.druid.query.SegmentDescriptor; -import io.druid.query.UnionQueryRunner; -import org.apache.commons.io.FileUtils; +import org.joda.time.DateTime; import org.joda.time.Interval; -import java.io.File; import java.util.Collection; +import java.util.List; import java.util.Set; import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentSkipListSet; +import java.util.concurrent.TimeUnit; /** * Runs tasks in a JVM thread using an ExecutorService. */ public class ThreadPoolTaskRunner implements TaskRunner, QuerySegmentWalker { + private static final EmittingLogger log = new EmittingLogger(ThreadPoolTaskRunner.class); + private final TaskToolboxFactory toolboxFactory; + private final TaskConfig taskConfig; private final ListeningExecutorService exec; private final Set runningItems = new ConcurrentSkipListSet<>(); - private final QueryRunnerFactoryConglomerate conglomerate; - private static final EmittingLogger log = new EmittingLogger(ThreadPoolTaskRunner.class); + private final ServiceEmitter emitter; @Inject public ThreadPoolTaskRunner( TaskToolboxFactory toolboxFactory, - QueryRunnerFactoryConglomerate conglomerate + TaskConfig taskConfig, + ServiceEmitter emitter ) { this.toolboxFactory = Preconditions.checkNotNull(toolboxFactory, "toolboxFactory"); + this.taskConfig = taskConfig; this.exec = MoreExecutors.listeningDecorator(Execs.singleThreaded("task-runner-%d")); - this.conglomerate = conglomerate; + this.emitter = Preconditions.checkNotNull(emitter, "emitter"); + } + + @Override + public List>> restore() + { + return ImmutableList.of(); } @LifecycleStop public void stop() { + exec.shutdown(); + + for (ThreadPoolTaskRunnerWorkItem item : runningItems) { + final Task task = item.getTask(); + final long start = System.currentTimeMillis(); + final boolean graceful; + final long elapsed; + boolean error = false; + + if (task.canRestore()) { + // Attempt graceful shutdown. + graceful = true; + log.info("Starting graceful shutdown of task[%s].", task.getId()); + + try { + task.stopGracefully(); + final TaskStatus taskStatus = item.getResult().get( + new Interval(new DateTime(start), taskConfig.getGracefulShutdownTimeout()).toDurationMillis(), + TimeUnit.MILLISECONDS + ); + log.info( + "Graceful shutdown of task[%s] finished in %,dms with status[%s].", + task.getId(), + System.currentTimeMillis() - start, + taskStatus.getStatusCode() + ); + } + catch (Exception e) { + log.makeAlert(e, "Graceful task shutdown failed: %s", task.getDataSource()) + .addData("taskId", task.getId()) + .addData("dataSource", task.getDataSource()) + .emit(); + log.warn(e, "Graceful shutdown of task[%s] aborted with exception."); + error = true; + } + } else { + graceful = false; + } + + elapsed = System.currentTimeMillis() - start; + + final ServiceMetricEvent.Builder metricBuilder = ServiceMetricEvent + .builder() + .setDimension("task", task.getId()) + .setDimension("dataSource", task.getDataSource()) + .setDimension("graceful", String.valueOf(graceful)) + .setDimension("error", String.valueOf(error)); + + emitter.emit(metricBuilder.build("task/interrupt/count", 1L)); + emitter.emit(metricBuilder.build("task/interrupt/elapsed", elapsed)); + } + + // Ok, now interrupt everything. exec.shutdownNow(); } @@ -212,7 +277,6 @@ public ThreadPoolTaskRunnerCallable(Task task, TaskToolbox toolbox) public TaskStatus call() { final long startTime = System.currentTimeMillis(); - final File taskDir = toolbox.getTaskWorkDir(); TaskStatus status; @@ -233,19 +297,6 @@ public TaskStatus call() throw Throwables.propagate(t); } - try { - if (taskDir.exists()) { - log.info("Removing task directory: %s", taskDir); - FileUtils.deleteDirectory(taskDir); - } - } - catch (Exception e) { - log.makeAlert(e, "Failed to delete task directory") - .addData("taskDir", taskDir.toString()) - .addData("task", task.getId()) - .emit(); - } - try { return status.withDuration(System.currentTimeMillis() - startTime); } diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/autoscaling/ec2/EC2AutoScaler.java b/indexing-service/src/main/java/io/druid/indexing/overlord/autoscaling/ec2/EC2AutoScaler.java index f0965b6c0980..ec02a4850f92 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/autoscaling/ec2/EC2AutoScaler.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/autoscaling/ec2/EC2AutoScaler.java @@ -32,6 +32,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Function; +import com.google.common.collect.FluentIterable; import com.google.common.collect.Lists; import com.metamx.emitter.EmittingLogger; import io.druid.indexing.overlord.autoscaling.AutoScaler; @@ -45,6 +46,7 @@ public class EC2AutoScaler implements AutoScaler { private static final EmittingLogger log = new EmittingLogger(EC2AutoScaler.class); + public static final int MAX_AWS_FILTER_VALUES = 100; private final int minNumWorkers; private final int maxNumWorkers; @@ -244,29 +246,35 @@ public AutoScalingData terminateWithIds(List ids) @Override public List ipToIdLookup(List ips) { - DescribeInstancesResult result = amazonEC2Client.describeInstances( - new DescribeInstancesRequest() - .withFilters( - new Filter("private-ip-address", ips) - ) - ); - - List instances = Lists.newArrayList(); - for (Reservation reservation : result.getReservations()) { - instances.addAll(reservation.getInstances()); - } - - List retVal = Lists.transform( - instances, - new Function() + final List retVal = FluentIterable + // chunk requests to avoid hitting default AWS limits on filters + .from(Lists.partition(ips, MAX_AWS_FILTER_VALUES)) + .transformAndConcat(new Function, Iterable>() { @Override - public String apply(Instance input) + public Iterable apply(List input) { - return input.getInstanceId(); + return amazonEC2Client.describeInstances( + new DescribeInstancesRequest().withFilters(new Filter("private-ip-address", input)) + ).getReservations(); } - } - ); + }) + .transformAndConcat(new Function>() + { + @Override + public Iterable apply(Reservation reservation) + { + return reservation.getInstances(); + } + }) + .transform(new Function() + { + @Override + public String apply(Instance instance) + { + return instance.getInstanceId(); + } + }).toList(); log.debug("Performing lookup: %s --> %s", ips, retVal); @@ -276,29 +284,35 @@ public String apply(Instance input) @Override public List idToIpLookup(List nodeIds) { - DescribeInstancesResult result = amazonEC2Client.describeInstances( - new DescribeInstancesRequest() - .withFilters( - new Filter("instance-id", nodeIds) - ) - ); - - List instances = Lists.newArrayList(); - for (Reservation reservation : result.getReservations()) { - instances.addAll(reservation.getInstances()); - } - - List retVal = Lists.transform( - instances, - new Function() + final List retVal = FluentIterable + // chunk requests to avoid hitting default AWS limits on filters + .from(Lists.partition(nodeIds, MAX_AWS_FILTER_VALUES)) + .transformAndConcat(new Function, Iterable>() { @Override - public String apply(Instance input) + public Iterable apply(List input) { - return input.getPrivateIpAddress(); + return amazonEC2Client.describeInstances( + new DescribeInstancesRequest().withFilters(new Filter("instance-id", input)) + ).getReservations(); } - } - ); + }) + .transformAndConcat(new Function>() + { + @Override + public Iterable apply(Reservation reservation) + { + return reservation.getInstances(); + } + }) + .transform(new Function() + { + @Override + public String apply(Instance instance) + { + return instance.getPrivateIpAddress(); + } + }).toList(); log.debug("Performing lookup: %s --> %s", nodeIds, retVal); diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/config/ForkingTaskRunnerConfig.java b/indexing-service/src/main/java/io/druid/indexing/overlord/config/ForkingTaskRunnerConfig.java index f35825b7cf27..182a028c85b9 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/config/ForkingTaskRunnerConfig.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/config/ForkingTaskRunnerConfig.java @@ -61,6 +61,13 @@ public class ForkingTaskRunnerConfig "hadoop" ); + @JsonProperty + private boolean separateIngestionEndpoint = false; + + public boolean isSeparateIngestionEndpoint() { + return separateIngestionEndpoint; + } + public String getJavaCommand() { return javaCommand; diff --git a/indexing-service/src/main/java/io/druid/indexing/worker/WorkerTaskMonitor.java b/indexing-service/src/main/java/io/druid/indexing/worker/WorkerTaskMonitor.java index 1ad9b5914d51..5dc96b3af6ff 100644 --- a/indexing-service/src/main/java/io/druid/indexing/worker/WorkerTaskMonitor.java +++ b/indexing-service/src/main/java/io/druid/indexing/worker/WorkerTaskMonitor.java @@ -18,7 +18,10 @@ package io.druid.indexing.worker; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.api.client.util.Sets; +import com.google.common.util.concurrent.ListenableFuture; import com.google.inject.Inject; +import com.metamx.common.Pair; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.emitter.EmittingLogger; @@ -28,18 +31,20 @@ import io.druid.indexing.overlord.TaskRunner; import io.druid.indexing.worker.config.WorkerConfig; import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.listen.Listenable; import org.apache.curator.framework.recipes.cache.PathChildrenCache; import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent; import org.apache.curator.framework.recipes.cache.PathChildrenCacheListener; import java.util.List; +import java.util.Set; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.ExecutorService; /** * The monitor watches ZK at a specified path for new tasks to appear. Upon starting the monitor, a listener will be * created that waits for new tasks. Tasks are executed as soon as they are seen. - * + *

* The monitor implements {@link io.druid.query.QuerySegmentWalker} so tasks can offer up queryable data. This is useful for * realtime index tasks. */ @@ -84,14 +89,20 @@ public WorkerTaskMonitor( public void start() { try { + // restore restorable tasks + final List>> restored = taskRunner.restore(); + for (Pair> pair : restored) { + submitTaskRunnable(pair.lhs, pair.rhs); + } + // cleanup any old running task announcements which are invalid after restart - for (TaskAnnouncement announcement : workerCuratorCoordinator.getAnnouncements()){ - if(announcement.getTaskStatus().isRunnable()) { + for (TaskAnnouncement announcement : workerCuratorCoordinator.getAnnouncements()) { + if (!isTaskRunning(announcement.getTaskStatus().getId()) && announcement.getTaskStatus().isRunnable()) { workerCuratorCoordinator.updateAnnouncement( TaskAnnouncement.create( - announcement.getTaskId(), + announcement.getTaskStatus().getId(), announcement.getTaskResource(), - TaskStatus.failure(announcement.getTaskId()) + TaskStatus.failure(announcement.getTaskStatus().getId()) ) ); } @@ -110,67 +121,7 @@ public void childEvent(CuratorFramework curatorFramework, PathChildrenCacheEvent Task.class ); - if (isTaskRunning(task)) { - log.warn( - "I can't build it. There's something in the way. Got task %s that I am already running...", - task.getId() - ); - workerCuratorCoordinator.unannounceTask(task.getId()); - return; - } - - log.info("Submitting runnable for task[%s]", task.getId()); - - exec.submit( - new Runnable() - { - @Override - public void run() - { - final long startTime = System.currentTimeMillis(); - - log.info("Affirmative. Running task [%s]", task.getId()); - running.add(task); - - TaskStatus taskStatus; - try { - workerCuratorCoordinator.unannounceTask(task.getId()); - workerCuratorCoordinator.announceTaskAnnouncement( - TaskAnnouncement.create( - task, - TaskStatus.running(task.getId()) - ) - ); - taskStatus = taskRunner.run(task).get(); - } - catch (Exception e) { - log.makeAlert(e, "I can't build there. Failed to run task") - .addData("task", task.getId()) - .emit(); - taskStatus = TaskStatus.failure(task.getId()); - } - finally { - running.remove(task); - } - - taskStatus = taskStatus.withDuration(System.currentTimeMillis() - startTime); - - try { - workerCuratorCoordinator.updateAnnouncement(TaskAnnouncement.create(task, taskStatus)); - log.info( - "Job's finished. Completed [%s] with status [%s]", - task.getId(), - taskStatus.getStatusCode() - ); - } - catch (Exception e) { - log.makeAlert(e, "Failed to update task status") - .addData("task", task.getId()) - .emit(); - } - } - } - ); + submitTaskRunnable(task, null); } } } @@ -184,10 +135,86 @@ public void run() } } - private boolean isTaskRunning(final Task task) + private void submitTaskRunnable(final Task task, final ListenableFuture taskStatusAlreadySubmitted) + { + if (isTaskRunning(task.getId())) { + log.warn( + "I can't build it. There's something in the way. Got task %s that I am already running...", + task.getId() + ); + workerCuratorCoordinator.unannounceTask(task.getId()); + return; + } + + log.info("Submitting runnable for task[%s]", task.getId()); + + running.add(task); + + exec.submit( + new Runnable() + { + @Override + public void run() + { + final long startTime = System.currentTimeMillis(); + + TaskStatus taskStatus; + + try { + workerCuratorCoordinator.updateAnnouncement( + TaskAnnouncement.create( + task, + TaskStatus.running(task.getId()) + ) + ); + + if (taskStatusAlreadySubmitted != null) { + log.info("Affirmative. Connecting to already-running task [%s]", task.getId()); + taskStatus = taskStatusAlreadySubmitted.get(); + } else { + log.info("Affirmative. Running task [%s]", task.getId()); + workerCuratorCoordinator.unannounceTask(task.getId()); + taskStatus = taskRunner.run(task).get(); + } + } + catch (InterruptedException e) { + log.debug(e, "Interrupted while running task[%s], exiting.", task.getId()); + return; + } + catch (Exception e) { + log.makeAlert(e, "I can't build there. Failed to run task") + .addData("task", task.getId()) + .emit(); + taskStatus = TaskStatus.failure(task.getId()); + } + finally { + running.remove(task); + } + + taskStatus = taskStatus.withDuration(System.currentTimeMillis() - startTime); + + try { + workerCuratorCoordinator.updateAnnouncement(TaskAnnouncement.create(task, taskStatus)); + log.info( + "Job's finished. Completed [%s] with status [%s]", + task.getId(), + taskStatus.getStatusCode() + ); + } + catch (Exception e) { + log.makeAlert(e, "Failed to update task status") + .addData("task", task.getId()) + .emit(); + } + } + } + ); + } + + private boolean isTaskRunning(final String taskId) { for (final Task runningTask : running) { - if (runningTask.getId().equals(task.getId())) { + if (runningTask.getId().equals(taskId)) { return true; } } @@ -200,7 +227,8 @@ public void stop() { try { pathChildrenCache.close(); - exec.shutdown(); + exec.shutdownNow(); + taskRunner.stop(); } catch (Exception e) { log.makeAlert(e, "Exception stopping WorkerTaskMonitor") diff --git a/indexing-service/src/main/java/io/druid/indexing/worker/executor/ExecutorLifecycle.java b/indexing-service/src/main/java/io/druid/indexing/worker/executor/ExecutorLifecycle.java index 2850293231ed..f3909a0bc61d 100644 --- a/indexing-service/src/main/java/io/druid/indexing/worker/executor/ExecutorLifecycle.java +++ b/indexing-service/src/main/java/io/druid/indexing/worker/executor/ExecutorLifecycle.java @@ -31,12 +31,17 @@ import io.druid.concurrent.Execs; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.actions.TaskActionClientFactory; +import io.druid.indexing.common.config.TaskConfig; import io.druid.indexing.common.task.Task; import io.druid.indexing.overlord.TaskRunner; +import org.joda.time.DateTime; import java.io.File; import java.io.IOException; import java.io.InputStream; +import java.nio.channels.FileChannel; +import java.nio.channels.FileLock; +import java.nio.file.StandardOpenOption; import java.util.concurrent.ExecutorService; /** @@ -47,37 +52,41 @@ public class ExecutorLifecycle { private static final EmittingLogger log = new EmittingLogger(ExecutorLifecycle.class); - private final ExecutorLifecycleConfig config; + private final ExecutorLifecycleConfig taskExecutorConfig; + private final TaskConfig taskConfig; private final TaskActionClientFactory taskActionClientFactory; private final TaskRunner taskRunner; private final ObjectMapper jsonMapper; private final ExecutorService parentMonitorExec = Execs.singleThreaded("parent-monitor-%d"); + private volatile Task task = null; private volatile ListenableFuture statusFuture = null; + private volatile FileChannel taskLockChannel; + private volatile FileLock taskLockFileLock; @Inject public ExecutorLifecycle( - ExecutorLifecycleConfig config, + ExecutorLifecycleConfig taskExecutorConfig, + TaskConfig taskConfig, TaskActionClientFactory taskActionClientFactory, TaskRunner taskRunner, ObjectMapper jsonMapper ) { - this.config = config; + this.taskExecutorConfig = taskExecutorConfig; + this.taskConfig = taskConfig; this.taskActionClientFactory = taskActionClientFactory; this.taskRunner = taskRunner; this.jsonMapper = jsonMapper; } @LifecycleStart - public void start() + public void start() throws InterruptedException { - final File taskFile = Preconditions.checkNotNull(config.getTaskFile(), "taskFile"); - final File statusFile = Preconditions.checkNotNull(config.getStatusFile(), "statusFile"); - final InputStream parentStream = Preconditions.checkNotNull(config.getParentStream(), "parentStream"); - - final Task task; + final File taskFile = Preconditions.checkNotNull(taskExecutorConfig.getTaskFile(), "taskFile"); + final File statusFile = Preconditions.checkNotNull(taskExecutorConfig.getStatusFile(), "statusFile"); + final InputStream parentStream = Preconditions.checkNotNull(taskExecutorConfig.getParentStream(), "parentStream"); try { task = jsonMapper.readValue(taskFile, Task.class); @@ -91,6 +100,43 @@ public void start() throw Throwables.propagate(e); } + // Avoid running the same task twice on the same machine by locking the task base directory. + + final File taskLockFile = taskConfig.getTaskLockFile(task.getId()); + + try { + synchronized (this) { + if (taskLockChannel == null && taskLockFileLock == null) { + taskLockChannel = FileChannel.open( + taskLockFile.toPath(), + StandardOpenOption.CREATE, + StandardOpenOption.WRITE + ); + + log.info("Attempting to lock file[%s].", taskLockFile); + final long startLocking = System.currentTimeMillis(); + final long timeout = new DateTime(startLocking).plus(taskConfig.getDirectoryLockTimeout()).getMillis(); + while (taskLockFileLock == null && System.currentTimeMillis() < timeout) { + taskLockFileLock = taskLockChannel.tryLock(); + if (taskLockFileLock == null) { + Thread.sleep(100); + } + } + + if (taskLockFileLock == null) { + throw new ISE("Could not acquire lock file[%s] within %,dms.", taskLockFile, timeout - startLocking); + } else { + log.info("Acquired lock file[%s] in %,dms.", taskLockFile, System.currentTimeMillis() - startLocking); + } + } else { + throw new ISE("Already started!"); + } + } + } + catch (IOException e) { + throw Throwables.propagate(e); + } + // Spawn monitor thread to keep a watch on parent's stdin // If stdin reaches eof, the parent is gone, and we should shut down parentMonitorExec.submit( @@ -120,7 +166,8 @@ public void run() if (!task.isReady(taskActionClientFactory.create(task))) { throw new ISE("Task is not ready to run yet!", task.getId()); } - } catch (Exception e) { + } + catch (Exception e) { throw new ISE(e, "Failed to run isReady", task.getId()); } @@ -164,8 +211,18 @@ public void join() } @LifecycleStop - public void stop() + public void stop() throws Exception { parentMonitorExec.shutdown(); + + synchronized (this) { + if (taskLockFileLock != null) { + taskLockFileLock.release(); + } + + if (taskLockChannel != null) { + taskLockChannel.close(); + } + } } } diff --git a/indexing-service/src/test/java/io/druid/indexing/common/TaskToolboxTest.java b/indexing-service/src/test/java/io/druid/indexing/common/TaskToolboxTest.java index 261d888334e5..ab46bd8ec44a 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/TaskToolboxTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/TaskToolboxTest.java @@ -24,6 +24,8 @@ import com.metamx.emitter.service.ServiceEmitter; import com.metamx.metrics.MonitorScheduler; import io.druid.client.FilteredServerView; +import io.druid.client.cache.Cache; +import io.druid.client.cache.CacheConfig; import io.druid.indexing.common.actions.TaskActionClientFactory; import io.druid.indexing.common.config.TaskConfig; import io.druid.indexing.common.task.Task; @@ -70,6 +72,8 @@ public class TaskToolboxTest private ObjectMapper ObjectMapper = new ObjectMapper(); private SegmentLoaderLocalCacheManager mockSegmentLoaderLocalCacheManager = EasyMock.createMock(SegmentLoaderLocalCacheManager.class); private Task task = EasyMock.createMock(Task.class); + private Cache mockCache = EasyMock.createMock(Cache.class); + private CacheConfig mockCacheConfig = EasyMock.createMock(CacheConfig.class); @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); @@ -81,7 +85,7 @@ public void setUp() throws IOException EasyMock.replay(task); taskToolbox = new TaskToolboxFactory( - new TaskConfig(temporaryFolder.newFile().toString(), null, null, 50000, null), + new TaskConfig(temporaryFolder.newFile().toString(), null, null, 50000, null, null, null), mockTaskActionClientFactory, mockEmitter, mockSegmentPusher, @@ -94,7 +98,9 @@ public void setUp() throws IOException mockQueryExecutorService, mockMonitorScheduler, new SegmentLoaderFactory(mockSegmentLoaderLocalCacheManager), - ObjectMapper + ObjectMapper, + mockCache, + mockCacheConfig ); } @@ -174,4 +180,16 @@ public void testGetDataSegmentMover() { Assert.assertEquals(mockDataSegmentMover, taskToolbox.build(task).getDataSegmentMover()); } + + @Test + public void testGetCache() throws Exception + { + Assert.assertEquals(mockCache, taskToolbox.build(task).getCache()); + } + + @Test + public void testGetCacheConfig() throws Exception + { + Assert.assertEquals(mockCacheConfig, taskToolbox.build(task).getCacheConfig()); + } } diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java index db30ddc4033f..d25b3fab60b5 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java @@ -243,7 +243,8 @@ public DataSegment push(File file, DataSegment segment) throws IOException segments.add(segment); return segment; } - }, null, null, null, null, null, null, null, null, null, null, temporaryFolder.newFolder() + }, null, null, null, null, null, null, null, null, null, null, temporaryFolder.newFolder(), + null, null ) ); diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java index 67e0f81b8530..49b62dcd4513 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java @@ -19,12 +19,138 @@ package io.druid.indexing.common.task; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.api.client.util.Charsets; +import com.google.api.client.util.Sets; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; +import com.metamx.common.Granularity; +import com.metamx.common.ISE; +import com.metamx.common.guava.Sequences; +import com.metamx.common.logger.Logger; +import com.metamx.emitter.EmittingLogger; +import com.metamx.emitter.core.LoggingEmitter; +import com.metamx.emitter.service.ServiceEmitter; +import com.metamx.metrics.MonitorScheduler; +import io.druid.client.cache.CacheConfig; +import io.druid.client.cache.MapCache; +import io.druid.concurrent.Execs; +import io.druid.data.input.InputRow; +import io.druid.data.input.MapBasedInputRow; +import io.druid.granularity.QueryGranularity; +import io.druid.indexing.common.SegmentLoaderFactory; +import io.druid.indexing.common.TaskStatus; +import io.druid.indexing.common.TaskToolbox; +import io.druid.indexing.common.TaskToolboxFactory; +import io.druid.indexing.common.TestUtils; +import io.druid.indexing.common.actions.LocalTaskActionClientFactory; +import io.druid.indexing.common.actions.TaskActionClientFactory; +import io.druid.indexing.common.actions.TaskActionToolbox; +import io.druid.indexing.common.config.TaskConfig; +import io.druid.indexing.common.config.TaskStorageConfig; +import io.druid.indexing.overlord.HeapMemoryTaskStorage; +import io.druid.indexing.overlord.IndexerMetadataStorageCoordinator; +import io.druid.indexing.overlord.TaskLockbox; +import io.druid.indexing.overlord.TaskStorage; +import io.druid.indexing.test.TestDataSegmentAnnouncer; +import io.druid.indexing.test.TestDataSegmentKiller; +import io.druid.indexing.test.TestDataSegmentPusher; +import io.druid.indexing.test.TestIndexerMetadataStorageCoordinator; +import io.druid.indexing.test.TestServerView; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.query.DefaultQueryRunnerFactoryConglomerate; +import io.druid.query.Druids; +import io.druid.query.IntervalChunkingQueryRunnerDecorator; +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.query.QueryRunnerFactory; +import io.druid.query.QueryRunnerFactoryConglomerate; +import io.druid.query.QueryToolChest; +import io.druid.query.QueryWatcher; +import io.druid.query.Result; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.CountAggregatorFactory; +import io.druid.query.aggregation.LongSumAggregatorFactory; +import io.druid.query.timeseries.TimeseriesQuery; +import io.druid.query.timeseries.TimeseriesQueryEngine; +import io.druid.query.timeseries.TimeseriesQueryQueryToolChest; +import io.druid.query.timeseries.TimeseriesQueryRunnerFactory; +import io.druid.query.timeseries.TimeseriesResultValue; +import io.druid.segment.indexing.DataSchema; +import io.druid.segment.indexing.RealtimeIOConfig; +import io.druid.segment.indexing.RealtimeTuningConfig; +import io.druid.segment.indexing.granularity.UniformGranularitySpec; +import io.druid.segment.loading.SegmentLoaderConfig; +import io.druid.segment.loading.SegmentLoaderLocalCacheManager; +import io.druid.segment.loading.StorageLocationConfig; +import io.druid.segment.realtime.FireDepartment; +import io.druid.segment.realtime.firehose.EventReceiverFirehoseFactory; +import io.druid.segment.realtime.firehose.NoopChatHandlerProvider; +import io.druid.server.coordination.DruidServerMetadata; +import io.druid.server.metrics.EventReceiverFirehoseRegister; +import io.druid.timeline.DataSegment; +import org.easymock.EasyMock; import org.joda.time.DateTime; +import org.joda.time.Period; +import org.junit.After; import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.Callable; public class RealtimeIndexTaskTest { + private static final Logger log = new Logger(RealtimeIndexTaskTest.class); + private static final DruidServerMetadata dummyServer = new DruidServerMetadata( + "dummy", + "dummy_host", + 0, + "historical", + "dummy_tier", + 0 + ); + private static final ObjectMapper jsonMapper = new DefaultObjectMapper(); + private static final ServiceEmitter emitter = new ServiceEmitter( + "service", + "host", + new LoggingEmitter( + log, + LoggingEmitter.Level.ERROR, + jsonMapper + ) + ); + + @Rule + public final TemporaryFolder tempFolder = new TemporaryFolder(); + + private DateTime now; + private ListeningExecutorService taskExec; + + @Before + public void setUp() + { + EmittingLogger.registerEmitter(emitter); + emitter.start(); + taskExec = MoreExecutors.listeningDecorator(Execs.singleThreaded("realtime-index-task-test-%d")); + now = new DateTime(); + } + + @After + public void tearDown() + { + taskExec.shutdownNow(); + } @Test public void testMakeTaskId() throws Exception @@ -34,4 +160,387 @@ public void testMakeTaskId() throws Exception RealtimeIndexTask.makeTaskId("test", 0, new DateTime("2015-01-02"), 0x76543210) ); } + + @Test(timeout = 10000L) + public void testBasics() throws Exception + { + final TestIndexerMetadataStorageCoordinator mdc = new TestIndexerMetadataStorageCoordinator(); + final RealtimeIndexTask task = makeRealtimeTask(null); + final TaskToolbox taskToolbox = makeToolbox(task, mdc, tempFolder.newFolder()); + final ListenableFuture statusFuture = runTask(task, taskToolbox); + + // Wait for firehose to show up, it starts off null. + while (task.getFirehose() == null) { + Thread.sleep(50); + } + + final EventReceiverFirehoseFactory.EventReceiverFirehose firehose = + (EventReceiverFirehoseFactory.EventReceiverFirehose) task.getFirehose(); + + firehose.addRows( + ImmutableList.of( + new MapBasedInputRow( + now, + ImmutableList.of("dim1"), + ImmutableMap.of("dim1", "foo") + ), + new MapBasedInputRow( + now, + ImmutableList.of("dim2"), + ImmutableMap.of("dim2", "bar") + ) + ) + ); + + // Stop the firehose, this will drain out existing events. + firehose.close(); + + // Wait for publish. + while (mdc.getPublished().isEmpty()) { + Thread.sleep(50); + } + + // Do a query. + Assert.assertEquals(2, countEvents(task)); + + // Simulate handoff. + for (DataSegment segment : mdc.getPublished()) { + ((TestServerView) taskToolbox.getNewSegmentServerView()).segmentAdded(dummyServer, segment); + } + + // Wait for the task to finish. + final TaskStatus taskStatus = statusFuture.get(); + Assert.assertEquals(TaskStatus.Status.SUCCESS, taskStatus.getStatusCode()); + } + + @Test(timeout = 10000L) + public void testRestore() throws Exception + { + final File directory = tempFolder.newFolder(); + final RealtimeIndexTask task1 = makeRealtimeTask(null); + + // First run: + { + final TestIndexerMetadataStorageCoordinator mdc = new TestIndexerMetadataStorageCoordinator(); + final TaskToolbox taskToolbox = makeToolbox(task1, mdc, directory); + final ListenableFuture statusFuture = runTask(task1, taskToolbox); + + // Wait for firehose to show up, it starts off null. + while (task1.getFirehose() == null) { + Thread.sleep(50); + } + + final EventReceiverFirehoseFactory.EventReceiverFirehose firehose = + (EventReceiverFirehoseFactory.EventReceiverFirehose) task1.getFirehose(); + + firehose.addRows( + ImmutableList.of( + new MapBasedInputRow( + now, + ImmutableList.of("dim1"), + ImmutableMap.of("dim1", "foo") + ) + ) + ); + + // Trigger graceful shutdown. + task1.stopGracefully(); + + // Wait for the task to finish. The status doesn't really matter, but we'll check it anyway. + final TaskStatus taskStatus = statusFuture.get(); + Assert.assertEquals(TaskStatus.Status.SUCCESS, taskStatus.getStatusCode()); + + // Nothing should be published. + Assert.assertEquals(Sets.newHashSet(), mdc.getPublished()); + } + + // Second run: + { + final TestIndexerMetadataStorageCoordinator mdc = new TestIndexerMetadataStorageCoordinator(); + final RealtimeIndexTask task2 = makeRealtimeTask(task1.getId()); + final TaskToolbox taskToolbox = makeToolbox(task2, mdc, directory); + final ListenableFuture statusFuture = runTask(task2, taskToolbox); + + // Wait for firehose to show up, it starts off null. + while (task2.getFirehose() == null) { + Thread.sleep(50); + } + + // Do a query, at this point the previous data should be loaded. + Assert.assertEquals(1, countEvents(task2)); + + final EventReceiverFirehoseFactory.EventReceiverFirehose firehose = + (EventReceiverFirehoseFactory.EventReceiverFirehose) task2.getFirehose(); + + firehose.addRows( + ImmutableList.of( + new MapBasedInputRow( + now, + ImmutableList.of("dim2"), + ImmutableMap.of("dim2", "bar") + ) + ) + ); + + // Stop the firehose, this will drain out existing events. + firehose.close(); + + // Wait for publish. + while (mdc.getPublished().isEmpty()) { + Thread.sleep(50); + } + + // Do a query. + Assert.assertEquals(2, countEvents(task2)); + + // Simulate handoff. + for (DataSegment segment : mdc.getPublished()) { + ((TestServerView) taskToolbox.getNewSegmentServerView()).segmentAdded(dummyServer, segment); + } + + // Wait for the task to finish. + final TaskStatus taskStatus = statusFuture.get(); + Assert.assertEquals(TaskStatus.Status.SUCCESS, taskStatus.getStatusCode()); + } + } + + @Test(timeout = 10000L) + public void testRestoreCorruptData() throws Exception + { + final File directory = tempFolder.newFolder(); + final RealtimeIndexTask task1 = makeRealtimeTask(null); + + // First run: + { + final TestIndexerMetadataStorageCoordinator mdc = new TestIndexerMetadataStorageCoordinator(); + final TaskToolbox taskToolbox = makeToolbox(task1, mdc, directory); + final ListenableFuture statusFuture = runTask(task1, taskToolbox); + + // Wait for firehose to show up, it starts off null. + while (task1.getFirehose() == null) { + Thread.sleep(50); + } + + final EventReceiverFirehoseFactory.EventReceiverFirehose firehose = + (EventReceiverFirehoseFactory.EventReceiverFirehose) task1.getFirehose(); + + firehose.addRows( + ImmutableList.of( + new MapBasedInputRow( + now, + ImmutableList.of("dim1"), + ImmutableMap.of("dim1", "foo") + ) + ) + ); + + // Trigger graceful shutdown. + task1.stopGracefully(); + + // Wait for the task to finish. The status doesn't really matter, but we'll check it anyway. + final TaskStatus taskStatus = statusFuture.get(); + Assert.assertEquals(TaskStatus.Status.SUCCESS, taskStatus.getStatusCode()); + + // Nothing should be published. + Assert.assertEquals(Sets.newHashSet(), mdc.getPublished()); + } + + // Corrupt the data: + final File smooshFile = new File( + String.format( + "%s/persistent/task/%s/work/persist/%s/%s_%s/0/00000.smoosh", + directory, + task1.getId(), + task1.getDataSource(), + Granularity.DAY.truncate(now), + Granularity.DAY.increment(Granularity.DAY.truncate(now)) + ) + ); + + Files.write(smooshFile.toPath(), "oops!".getBytes(Charsets.UTF_8)); + + // Second run: + { + final TestIndexerMetadataStorageCoordinator mdc = new TestIndexerMetadataStorageCoordinator(); + final RealtimeIndexTask task2 = makeRealtimeTask(task1.getId()); + final TaskToolbox taskToolbox = makeToolbox(task2, mdc, directory); + final ListenableFuture statusFuture = runTask(task2, taskToolbox); + + // Wait for the task to finish. + boolean caught = false; + try { + statusFuture.get(); + } + catch (Exception e) { + caught = true; + } + Assert.assertTrue("expected exception", caught); + } + } + + private ListenableFuture runTask(final Task task, final TaskToolbox toolbox) + { + return taskExec.submit( + new Callable() + { + @Override + public TaskStatus call() throws Exception + { + try { + if (task.isReady(toolbox.getTaskActionClient())) { + return task.run(toolbox); + } else { + throw new ISE("Task is not ready"); + } + } + catch (Exception e) { + log.warn(e, "Task failed"); + throw e; + } + } + } + ); + } + + private RealtimeIndexTask makeRealtimeTask(final String taskId) + { + ObjectMapper objectMapper = new DefaultObjectMapper(); + DataSchema dataSchema = new DataSchema( + "test_ds", + null, + new AggregatorFactory[]{new CountAggregatorFactory("rows")}, + new UniformGranularitySpec(Granularity.DAY, QueryGranularity.NONE, null), + objectMapper + ); + RealtimeIOConfig realtimeIOConfig = new RealtimeIOConfig( + new EventReceiverFirehoseFactory( + "foo", + 100, + new NoopChatHandlerProvider(), + objectMapper, + null, + new EventReceiverFirehoseRegister() + ), + null, + null + ); + RealtimeTuningConfig realtimeTuningConfig = new RealtimeTuningConfig( + 1000, + new Period("P1Y"), + new Period("PT10M"), + null, + null, + null, + null, + null, + null, + null, + null, + null, + null + ); + return new RealtimeIndexTask( + taskId, + null, + new FireDepartment(dataSchema, realtimeIOConfig, realtimeTuningConfig), + null + ); + } + + private TaskToolbox makeToolbox(final Task task, final IndexerMetadataStorageCoordinator mdc, final File directory) + { + final TaskStorage taskStorage = new HeapMemoryTaskStorage(new TaskStorageConfig(null)); + final TaskConfig taskConfig = new TaskConfig(directory.getPath(), null, null, 50000, null, null, null); + final TaskLockbox taskLockbox = new TaskLockbox(taskStorage); + final TaskActionToolbox taskActionToolbox = new TaskActionToolbox( + taskLockbox, + mdc, + emitter + ); + final TaskActionClientFactory taskActionClientFactory = new LocalTaskActionClientFactory( + taskStorage, + taskActionToolbox + ); + final QueryRunnerFactoryConglomerate conglomerate = new DefaultQueryRunnerFactoryConglomerate( + ImmutableMap., QueryRunnerFactory>of( + TimeseriesQuery.class, + new TimeseriesQueryRunnerFactory( + new TimeseriesQueryQueryToolChest( + new IntervalChunkingQueryRunnerDecorator(null, null, null) + { + @Override + public QueryRunner decorate( + QueryRunner delegate, QueryToolChest> toolChest + ) + { + return delegate; + } + } + ), + new TimeseriesQueryEngine(), + new QueryWatcher() + { + @Override + public void registerQuery(Query query, ListenableFuture future) + { + // do nothing + } + } + ) + ) + ); + final TaskToolboxFactory toolboxFactory = new TaskToolboxFactory( + taskConfig, + taskActionClientFactory, + emitter, + new TestDataSegmentPusher(), + new TestDataSegmentKiller(), + null, // DataSegmentMover + null, // DataSegmentArchiver + new TestDataSegmentAnnouncer(), + new TestServerView(), + conglomerate, + MoreExecutors.sameThreadExecutor(), // queryExecutorService + EasyMock.createMock(MonitorScheduler.class), + new SegmentLoaderFactory( + new SegmentLoaderLocalCacheManager( + null, + new SegmentLoaderConfig() + { + @Override + public List getLocations() + { + return Lists.newArrayList(); + } + }, TestUtils.MAPPER + ) + ), + TestUtils.MAPPER, + MapCache.create(1024), + new CacheConfig() + ); + + taskLockbox.add(task); + return toolboxFactory.build(task); + } + + public long countEvents(final Task task) throws Exception + { + // Do a query. + TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() + .dataSource("test_ds") + .aggregators( + ImmutableList.of( + new LongSumAggregatorFactory("rows", "rows") + ) + ).granularity(QueryGranularity.ALL) + .intervals("2000/3000") + .build(); + + ArrayList> results = Sequences.toList( + task.getQueryRunner(query).run(query, ImmutableMap.of()), + Lists.>newArrayList() + ); + return results.get(0).getValue().getLongMetric("rows"); + } } diff --git a/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java b/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java index 43603cd5089b..fe418f9ca844 100644 --- a/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java @@ -181,7 +181,7 @@ public void deleteSegments(Set segments) ); final ObjectMapper objectMapper = newObjectMapper(); final TaskToolboxFactory taskToolboxFactory = new TaskToolboxFactory( - new TaskConfig(tmpDir.getAbsolutePath(), null, null, 50000, null), + new TaskConfig(tmpDir.getAbsolutePath(), null, null, 50000, null, null, null), tac, newMockEmitter(), new DataSegmentPusher() @@ -247,7 +247,9 @@ public List getLocations() }, objectMapper ) ), - objectMapper + objectMapper, + null, + null ); Collection values = new LinkedList<>(); for (InputRowParser parser : Arrays.asList( diff --git a/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java b/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java index 5f5f371b55d9..d49910b68704 100644 --- a/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java @@ -282,7 +282,7 @@ public RetType submit(TaskAction taskAction) throws IOExcepti } }; final TaskToolboxFactory taskToolboxFactory = new TaskToolboxFactory( - new TaskConfig(testCase.tmpDir.getAbsolutePath(), null, null, 50000, null), + new TaskConfig(testCase.tmpDir.getAbsolutePath(), null, null, 50000, null, null, null), new TaskActionClientFactory() { @Override @@ -314,7 +314,9 @@ public List getLocations() }, objectMapper ) ), - objectMapper + objectMapper, + null, + null ); final Injector injector = Guice.createInjector( new Module() diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java index 606c8cc2f363..843500682ede 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java @@ -33,7 +33,6 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Ordering; -import com.google.common.collect.Sets; import com.metamx.common.Granularity; import com.metamx.common.ISE; import com.metamx.common.guava.Comparators; @@ -45,6 +44,7 @@ import com.metamx.metrics.MonitorScheduler; import io.druid.client.FilteredServerView; import io.druid.client.ServerView; +import io.druid.client.cache.MapCache; import io.druid.data.input.Firehose; import io.druid.data.input.FirehoseFactory; import io.druid.data.input.InputRow; @@ -68,11 +68,12 @@ import io.druid.indexing.common.task.IndexTask; import io.druid.indexing.common.task.KillTask; import io.druid.indexing.common.task.RealtimeIndexTask; +import io.druid.indexing.common.task.RealtimeIndexTaskTest; import io.druid.indexing.common.task.Task; import io.druid.indexing.common.task.TaskResource; import io.druid.indexing.overlord.config.TaskQueueConfig; +import io.druid.indexing.test.TestIndexerMetadataStorageCoordinator; import io.druid.jackson.DefaultObjectMapper; -import io.druid.metadata.IndexerSQLMetadataStorageCoordinator; import io.druid.metadata.SQLMetadataStorageActionHandlerFactory; import io.druid.metadata.TestDerbyConnector; import io.druid.query.QueryRunnerFactoryConglomerate; @@ -93,6 +94,7 @@ import io.druid.segment.loading.SegmentLoadingException; import io.druid.segment.loading.StorageLocationConfig; import io.druid.segment.realtime.FireDepartment; +import io.druid.segment.realtime.FireDepartmentTest; import io.druid.server.coordination.DataSegmentAnnouncer; import io.druid.server.coordination.DruidServerMetadata; import io.druid.timeline.DataSegment; @@ -169,6 +171,7 @@ public int compare(DataSegment dataSegment, DataSegment dataSegment2) private final String taskStorageType; + private ObjectMapper mapper; private TaskStorageQueryAdapter tsqa = null; private File tmpDir = null; @@ -176,7 +179,7 @@ public int compare(DataSegment dataSegment, DataSegment dataSegment2) private TaskLockbox tl = null; private TaskQueue tq = null; private TaskRunner tr = null; - private MockIndexerMetadataStorageCoordinator mdc = null; + private TestIndexerMetadataStorageCoordinator mdc = null; private TaskActionClientFactory tac = null; private TaskToolboxFactory tb = null; private IndexSpec indexSpec; @@ -191,9 +194,18 @@ public int compare(DataSegment dataSegment, DataSegment dataSegment2) private TestDerbyConnector testDerbyConnector; private List segmentCallbacks = new ArrayList<>(); - private static MockIndexerMetadataStorageCoordinator newMockMDC() + private static TestIndexerMetadataStorageCoordinator newMockMDC() { - return new MockIndexerMetadataStorageCoordinator(); + return new TestIndexerMetadataStorageCoordinator() + { + @Override + public Set announceHistoricalSegments(Set segments) + { + Set retVal = super.announceHistoricalSegments(segments); + publishCountDown.countDown(); + return retVal; + } + }; } private static ServiceEmitter newMockEmitter() @@ -359,7 +371,11 @@ public void setUp() throws Exception ts = new MetadataTaskStorage( testDerbyConnector, new TaskStorageConfig(null), - new SQLMetadataStorageActionHandlerFactory(testDerbyConnector, derbyConnectorRule.metadataTablesConfigSupplier().get(), mapper) + new SQLMetadataStorageActionHandlerFactory( + testDerbyConnector, + derbyConnectorRule.metadataTablesConfigSupplier().get(), + mapper + ) ); } else { throw new RuntimeException(String.format("Unknown task storage type [%s]", taskStorageType)); @@ -394,13 +410,15 @@ public DataSegment push(File file, DataSegment segment) throws IOException ); } - private void setUpAndStartTaskQueue(DataSegmentPusher dataSegmentPusher) { + private void setUpAndStartTaskQueue(DataSegmentPusher dataSegmentPusher) + { + final TaskConfig taskConfig = new TaskConfig(tmpDir.toString(), null, null, 50000, null, null, null); tsqa = new TaskStorageQueryAdapter(ts); tl = new TaskLockbox(ts); mdc = newMockMDC(); tac = new LocalTaskActionClientFactory(ts, new TaskActionToolbox(tl, mdc, newMockEmitter())); tb = new TaskToolboxFactory( - new TaskConfig(tmpDir.toString(), null, null, 50000, null), + taskConfig, tac, newMockEmitter(), dataSegmentPusher, @@ -471,9 +489,11 @@ public List getLocations() }, new DefaultObjectMapper() ) ), - new DefaultObjectMapper() + new DefaultObjectMapper(), + MapCache.create(0), + FireDepartmentTest.NO_CACHE_CONFIG ); - tr = new ThreadPoolTaskRunner(tb, null); + tr = new ThreadPoolTaskRunner(tb, taskConfig, emitter); tq = new TaskQueue(tqc, ts, tr, tac, tl, emitter); tq.start(); } @@ -803,7 +823,7 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception Assert.assertEquals("segments nuked", 0, mdc.getNuked().size()); } - @Test (timeout = 4000L) + @Test(timeout = 4000L) public void testRealtimeIndexTask() throws Exception { monitorScheduler.addMonitor(EasyMock.anyObject(Monitor.class)); @@ -852,7 +872,7 @@ public void testRealtimeIndexTask() throws Exception EasyMock.verify(monitorScheduler, queryRunnerFactoryConglomerate); } - @Test (timeout = 4000L) + @Test(timeout = 4000L) public void testRealtimeIndexTaskFailure() throws Exception { setUpAndStartTaskQueue( @@ -994,7 +1014,8 @@ private TaskStatus runTask(final Task task) throws Exception return retVal; } - private RealtimeIndexTask giveMeARealtimeIndexTask() { + private RealtimeIndexTask giveMeARealtimeIndexTask() + { String taskId = String.format("rt_task_%s", System.currentTimeMillis()); DataSchema dataSchema = new DataSchema( "test_ds", @@ -1005,7 +1026,8 @@ private RealtimeIndexTask giveMeARealtimeIndexTask() { ); RealtimeIOConfig realtimeIOConfig = new RealtimeIOConfig( new MockFirehoseFactory(true), - null, // PlumberSchool - Realtime Index Task always uses RealtimePlumber which is hardcoded in RealtimeIndexTask class + null, + // PlumberSchool - Realtime Index Task always uses RealtimePlumber which is hardcoded in RealtimeIndexTask class null ); RealtimeTuningConfig realtimeTuningConfig = new RealtimeTuningConfig( @@ -1031,64 +1053,4 @@ private RealtimeIndexTask giveMeARealtimeIndexTask() { null ); } - - private static class MockIndexerMetadataStorageCoordinator extends IndexerSQLMetadataStorageCoordinator - { - final private Set published = Sets.newHashSet(); - final private Set nuked = Sets.newHashSet(); - - private List unusedSegments; - - private MockIndexerMetadataStorageCoordinator() - { - super(null, null, null); - unusedSegments = Lists.newArrayList(); - } - - @Override - public List getUsedSegmentsForInterval(String dataSource, Interval interval) throws IOException - { - return ImmutableList.of(); - } - - @Override - public List getUnusedSegmentsForInterval(String dataSource, Interval interval) - { - return unusedSegments; - } - - @Override - public Set announceHistoricalSegments(Set segments) - { - Set added = Sets.newHashSet(); - for (final DataSegment segment : segments) { - if (published.add(segment)) { - added.add(segment); - } - } - TaskLifecycleTest.publishCountDown.countDown(); - return ImmutableSet.copyOf(added); - } - - @Override - public void deleteSegments(Set segments) - { - nuked.addAll(segments); - } - - public Set getPublished() - { - return ImmutableSet.copyOf(published); - } - - public Set getNuked() - { - return ImmutableSet.copyOf(nuked); - } - - public void setUnusedSegments(List unusedSegments) - { - this.unusedSegments = unusedSegments; - } - } } diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/autoscaling/EC2AutoScalerTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/autoscaling/EC2AutoScalerTest.java index 2861fcfcb237..2bcfeb03b2dd 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/autoscaling/EC2AutoScalerTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/autoscaling/EC2AutoScalerTest.java @@ -20,15 +20,17 @@ import com.amazonaws.services.ec2.AmazonEC2Client; import com.amazonaws.services.ec2.model.DescribeInstancesRequest; import com.amazonaws.services.ec2.model.DescribeInstancesResult; +import com.amazonaws.services.ec2.model.Filter; import com.amazonaws.services.ec2.model.Instance; import com.amazonaws.services.ec2.model.Reservation; import com.amazonaws.services.ec2.model.RunInstancesRequest; import com.amazonaws.services.ec2.model.RunInstancesResult; import com.amazonaws.services.ec2.model.TerminateInstancesRequest; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Charsets; +import com.google.common.base.Functions; +import com.google.common.collect.ContiguousSet; +import com.google.common.collect.DiscreteDomain; import com.google.common.collect.Lists; -import com.google.common.io.BaseEncoding; +import com.google.common.collect.Range; import io.druid.indexing.overlord.autoscaling.ec2.EC2AutoScaler; import io.druid.indexing.overlord.autoscaling.ec2.EC2EnvironmentConfig; import io.druid.indexing.overlord.autoscaling.ec2.EC2NodeData; @@ -41,7 +43,9 @@ import org.junit.Test; import java.util.Arrays; +import java.util.Collections; import java.util.Date; +import java.util.List; /** */ @@ -49,10 +53,14 @@ public class EC2AutoScalerTest { private static final String AMI_ID = "dummy"; private static final String INSTANCE_ID = "theInstance"; + public static final EC2EnvironmentConfig ENV_CONFIG = new EC2EnvironmentConfig( + "us-east-1a", + new EC2NodeData(AMI_ID, INSTANCE_ID, 1, 1, Lists.newArrayList(), "foo", "mySubnet", null, null), + new GalaxyEC2UserData(new DefaultObjectMapper(), "env", "version", "type") + ); private static final String IP = "dummyIP"; private AmazonEC2Client amazonEC2Client; - private RunInstancesResult runInstancesResult; private DescribeInstancesResult describeInstancesResult; private Reservation reservation; private Instance instance; @@ -62,7 +70,6 @@ public class EC2AutoScalerTest public void setUp() throws Exception { amazonEC2Client = EasyMock.createMock(AmazonEC2Client.class); - runInstancesResult = EasyMock.createMock(RunInstancesResult.class); describeInstancesResult = EasyMock.createMock(DescribeInstancesResult.class); reservation = EasyMock.createMock(Reservation.class); @@ -79,7 +86,6 @@ public void setUp() throws Exception public void tearDown() throws Exception { EasyMock.verify(amazonEC2Client); - EasyMock.verify(runInstancesResult); EasyMock.verify(describeInstancesResult); EasyMock.verify(reservation); } @@ -87,14 +93,12 @@ public void tearDown() throws Exception @Test public void testScale() { + RunInstancesResult runInstancesResult = EasyMock.createMock(RunInstancesResult.class); + EC2AutoScaler autoScaler = new EC2AutoScaler( 0, 1, - new EC2EnvironmentConfig( - "us-east-1a", - new EC2NodeData(AMI_ID, INSTANCE_ID, 1, 1, Lists.newArrayList(), "foo", "mySubnet", null, null), - new GalaxyEC2UserData(new DefaultObjectMapper(), "env", "version", "type") - ), + ENV_CONFIG, amazonEC2Client, managementConfig ); @@ -111,10 +115,10 @@ public void testScale() EasyMock.expect(runInstancesResult.getReservation()).andReturn(reservation).atLeastOnce(); EasyMock.replay(runInstancesResult); - EasyMock.expect(describeInstancesResult.getReservations()).andReturn(Arrays.asList(reservation)).atLeastOnce(); + EasyMock.expect(describeInstancesResult.getReservations()).andReturn(Collections.singletonList(reservation)).atLeastOnce(); EasyMock.replay(describeInstancesResult); - EasyMock.expect(reservation.getInstances()).andReturn(Arrays.asList(instance)).atLeastOnce(); + EasyMock.expect(reservation.getInstances()).andReturn(Collections.singletonList(instance)).atLeastOnce(); EasyMock.replay(reservation); AutoScalingData created = autoScaler.provision(); @@ -126,5 +130,121 @@ public void testScale() Assert.assertEquals(deleted.getNodeIds().size(), 1); Assert.assertEquals(INSTANCE_ID, deleted.getNodeIds().get(0)); + + EasyMock.verify(runInstancesResult); + } + + @Test + public void testIptoIdLookup() throws Exception + { + EC2AutoScaler autoScaler = new EC2AutoScaler( + 0, + 1, + ENV_CONFIG, + amazonEC2Client, + managementConfig + ); + + final int n = 150; + Assert.assertTrue(n <= 2 * EC2AutoScaler.MAX_AWS_FILTER_VALUES); + + List ips = Lists.transform( + ContiguousSet.create(Range.closedOpen(0, n), DiscreteDomain.integers()).asList(), + Functions.toStringFunction() + ); + + EasyMock.expect(amazonEC2Client.describeInstances( + new DescribeInstancesRequest().withFilters(new Filter( + "private-ip-address", + ips.subList(0, EC2AutoScaler.MAX_AWS_FILTER_VALUES) + )) + )) + .andReturn(describeInstancesResult); + + EasyMock.expect(amazonEC2Client.describeInstances( + new DescribeInstancesRequest().withFilters(new Filter( + "private-ip-address", + ips.subList(EC2AutoScaler.MAX_AWS_FILTER_VALUES, n) + )) + )) + .andReturn(describeInstancesResult); + + EasyMock.replay(amazonEC2Client); + + final Reservation[] chunk1 = new Reservation[EC2AutoScaler.MAX_AWS_FILTER_VALUES]; + Arrays.fill(chunk1, reservation); + final Reservation[] chunk2 = new Reservation[n - EC2AutoScaler.MAX_AWS_FILTER_VALUES]; + Arrays.fill(chunk2, reservation); + EasyMock.expect(describeInstancesResult.getReservations()).andReturn( + Lists.newArrayList(chunk1) + ); + EasyMock.expect(describeInstancesResult.getReservations()).andReturn( + Lists.newArrayList(chunk2) + ); + EasyMock.replay(describeInstancesResult); + + EasyMock.expect(reservation.getInstances()).andReturn(Arrays.asList(instance)).times(n); + EasyMock.replay(reservation); + + List ids = autoScaler.ipToIdLookup(ips); + + Assert.assertEquals(n, ids.size()); + } + + @Test + public void testIdToIpLookup() throws Exception + { + EC2AutoScaler autoScaler = new EC2AutoScaler( + 0, + 1, + ENV_CONFIG, + amazonEC2Client, + managementConfig + ); + + final int n = 150; + Assert.assertTrue(n <= 2 * EC2AutoScaler.MAX_AWS_FILTER_VALUES); + + List ids = Lists.transform( + ContiguousSet.create(Range.closedOpen(0, n), DiscreteDomain.integers()).asList(), + Functions.toStringFunction() + ); + + EasyMock.expect(amazonEC2Client.describeInstances( + new DescribeInstancesRequest().withFilters(new Filter( + "instance-id", + ids.subList(0, EC2AutoScaler.MAX_AWS_FILTER_VALUES) + )) + )) + .andReturn(describeInstancesResult); + + EasyMock.expect(amazonEC2Client.describeInstances( + new DescribeInstancesRequest().withFilters(new Filter( + "instance-id", + ids.subList(EC2AutoScaler.MAX_AWS_FILTER_VALUES, n) + )) + )) + .andReturn(describeInstancesResult); + + EasyMock.replay(amazonEC2Client); + + final Reservation[] chunk1 = new Reservation[EC2AutoScaler.MAX_AWS_FILTER_VALUES]; + Arrays.fill(chunk1, reservation); + final Reservation[] chunk2 = new Reservation[n - EC2AutoScaler.MAX_AWS_FILTER_VALUES]; + Arrays.fill(chunk2, reservation); + EasyMock.expect(describeInstancesResult.getReservations()).andReturn( + Lists.newArrayList(chunk1) + ); + EasyMock.expect(describeInstancesResult.getReservations()).andReturn( + Lists.newArrayList(chunk2) + ); + EasyMock.replay(describeInstancesResult); + + EasyMock.expect(reservation.getInstances()).andReturn(Collections.singletonList(instance)).times(n); + EasyMock.replay(reservation); + + List ips = autoScaler.idToIpLookup(ids); + + Assert.assertEquals(n, ips.size()); } } diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordResourceTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordResourceTest.java index ec20523fe7bd..a28c332491f5 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordResourceTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordResourceTest.java @@ -25,6 +25,7 @@ import com.google.common.collect.Lists; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.MoreExecutors; +import com.metamx.common.Pair; import com.metamx.common.concurrent.ScheduledExecutorFactory; import com.metamx.common.guava.CloseQuietly; import com.metamx.emitter.EmittingLogger; @@ -294,6 +295,18 @@ public MockTaskRunner(CountDownLatch[] runLatches, CountDownLatch[] completionLa this.runningTasks = new ArrayList<>(); } + @Override + public List>> restore() + { + return ImmutableList.of(); + } + + @Override + public void stop() + { + // Do nothing + } + @Override public synchronized ListenableFuture run(final Task task) { diff --git a/indexing-service/src/test/java/io/druid/indexing/test/TestDataSegmentAnnouncer.java b/indexing-service/src/test/java/io/druid/indexing/test/TestDataSegmentAnnouncer.java new file mode 100644 index 000000000000..be5ef6df9ea2 --- /dev/null +++ b/indexing-service/src/test/java/io/druid/indexing/test/TestDataSegmentAnnouncer.java @@ -0,0 +1,66 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.test; + +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Sets; +import io.druid.server.coordination.DataSegmentAnnouncer; +import io.druid.timeline.DataSegment; + +import java.io.IOException; +import java.util.Set; + +public class TestDataSegmentAnnouncer implements DataSegmentAnnouncer +{ + public Set announcedSegments = Sets.newConcurrentHashSet(); + + @Override + public void announceSegment(DataSegment segment) throws IOException + { + announcedSegments.add(segment); + } + + @Override + public void unannounceSegment(DataSegment segment) throws IOException + { + announcedSegments.remove(segment); + } + + @Override + public void announceSegments(Iterable segments) throws IOException + { + for (DataSegment segment : segments) { + announcedSegments.add(segment); + } + } + + @Override + public void unannounceSegments(Iterable segments) throws IOException + { + for (DataSegment segment : segments) { + announcedSegments.remove(segment); + } + } + + public Set getAnnouncedSegments() + { + return ImmutableSet.copyOf(announcedSegments); + } +} diff --git a/indexing-service/src/test/java/io/druid/indexing/test/TestDataSegmentKiller.java b/indexing-service/src/test/java/io/druid/indexing/test/TestDataSegmentKiller.java new file mode 100644 index 000000000000..ab5971a0c07a --- /dev/null +++ b/indexing-service/src/test/java/io/druid/indexing/test/TestDataSegmentKiller.java @@ -0,0 +1,44 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.test; + +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Sets; +import io.druid.segment.loading.DataSegmentKiller; +import io.druid.segment.loading.SegmentLoadingException; +import io.druid.timeline.DataSegment; + +import java.util.Set; + +public class TestDataSegmentKiller implements DataSegmentKiller +{ + private final Set killedSegments = Sets.newConcurrentHashSet(); + + @Override + public void kill(DataSegment segment) throws SegmentLoadingException + { + killedSegments.add(segment); + } + + public Set getKilledSegments() + { + return ImmutableSet.copyOf(killedSegments); + } +} diff --git a/indexing-service/src/test/java/io/druid/indexing/test/TestDataSegmentPusher.java b/indexing-service/src/test/java/io/druid/indexing/test/TestDataSegmentPusher.java new file mode 100644 index 000000000000..b9bda3b6884c --- /dev/null +++ b/indexing-service/src/test/java/io/druid/indexing/test/TestDataSegmentPusher.java @@ -0,0 +1,52 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.test; + +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Sets; +import io.druid.segment.loading.DataSegmentPusher; +import io.druid.timeline.DataSegment; + +import java.io.File; +import java.io.IOException; +import java.util.Set; + +public class TestDataSegmentPusher implements DataSegmentPusher +{ + private final Set pushedSegments = Sets.newConcurrentHashSet(); + + @Override + public String getPathForHadoop(String dataSource) + { + throw new UnsupportedOperationException(); + } + + @Override + public DataSegment push(File file, DataSegment segment) throws IOException + { + pushedSegments.add(segment); + return segment; + } + + public Set getPushedSegments() + { + return ImmutableSet.copyOf(pushedSegments); + } +} diff --git a/indexing-service/src/test/java/io/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java b/indexing-service/src/test/java/io/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java new file mode 100644 index 000000000000..96a94bc4f9e1 --- /dev/null +++ b/indexing-service/src/test/java/io/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java @@ -0,0 +1,100 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.test; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import io.druid.indexing.overlord.IndexerMetadataStorageCoordinator; +import io.druid.timeline.DataSegment; +import org.joda.time.Interval; + +import java.io.IOException; +import java.util.List; +import java.util.Set; + +public class TestIndexerMetadataStorageCoordinator implements IndexerMetadataStorageCoordinator +{ + final private Set published = Sets.newConcurrentHashSet(); + final private Set nuked = Sets.newConcurrentHashSet(); + final private List unusedSegments; + + public TestIndexerMetadataStorageCoordinator() + { + unusedSegments = Lists.newArrayList(); + } + + @Override + public List getUsedSegmentsForInterval(String dataSource, Interval interval) throws IOException + { + return ImmutableList.of(); + } + + @Override + public List getUnusedSegmentsForInterval(String dataSource, Interval interval) + { + synchronized (unusedSegments) { + return ImmutableList.copyOf(unusedSegments); + } + } + + @Override + public Set announceHistoricalSegments(Set segments) + { + Set added = Sets.newHashSet(); + for (final DataSegment segment : segments) { + if (published.add(segment)) { + added.add(segment); + } + } + return ImmutableSet.copyOf(added); + } + + @Override + public void deleteSegments(Set segments) + { + nuked.addAll(segments); + } + + @Override + public void updateSegmentMetadata(Set segments) throws IOException + { + throw new UnsupportedOperationException(); + } + + public Set getPublished() + { + return ImmutableSet.copyOf(published); + } + + public Set getNuked() + { + return ImmutableSet.copyOf(nuked); + } + + public void setUnusedSegments(List newUnusedSegments) + { + synchronized (unusedSegments) { + unusedSegments.clear(); + unusedSegments.addAll(newUnusedSegments); + } + } +} diff --git a/indexing-service/src/test/java/io/druid/indexing/test/TestServerView.java b/indexing-service/src/test/java/io/druid/indexing/test/TestServerView.java new file mode 100644 index 000000000000..b9185547e096 --- /dev/null +++ b/indexing-service/src/test/java/io/druid/indexing/test/TestServerView.java @@ -0,0 +1,116 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexing.test; + +import com.google.api.client.util.Lists; +import com.google.common.base.Predicate; +import com.google.common.collect.Maps; +import com.metamx.common.Pair; +import io.druid.client.FilteredServerView; +import io.druid.client.ServerView; +import io.druid.server.coordination.DruidServerMetadata; +import io.druid.timeline.DataSegment; + +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.Executor; + +public class TestServerView implements FilteredServerView, ServerView.SegmentCallback +{ + final ConcurrentMap, Executor>> callbacks = Maps.newConcurrentMap(); + + @Override + public void registerSegmentCallback( + final Executor exec, + final ServerView.SegmentCallback callback, + final Predicate filter + ) + { + callbacks.put(callback, Pair.of(filter, exec)); + } + + @Override + public ServerView.CallbackAction segmentAdded( + final DruidServerMetadata server, + final DataSegment segment + ) + { + for (final Map.Entry, Executor>> entry : callbacks.entrySet()) { + if (entry.getValue().lhs.apply(segment)) { + entry.getValue().rhs.execute( + new Runnable() + { + @Override + public void run() + { + entry.getKey().segmentAdded(server, segment); + } + } + ); + } + } + + return ServerView.CallbackAction.CONTINUE; + } + + @Override + public ServerView.CallbackAction segmentRemoved( + final DruidServerMetadata server, + final DataSegment segment + ) + { + for (final Map.Entry, Executor>> entry : callbacks.entrySet()) { + if (entry.getValue().lhs.apply(segment)) { + entry.getValue().rhs.execute( + new Runnable() + { + @Override + public void run() + { + entry.getKey().segmentRemoved(server, segment); + } + } + ); + } + } + + return ServerView.CallbackAction.CONTINUE; + } + + @Override + public ServerView.CallbackAction segmentViewInitialized() + { + for (final Map.Entry, Executor>> entry : callbacks.entrySet()) { + entry.getValue().rhs.execute( + new Runnable() + { + @Override + public void run() + { + entry.getKey().segmentViewInitialized(); + } + } + ); + } + + return ServerView.CallbackAction.CONTINUE; + } +} diff --git a/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java b/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java index c4c447206bd3..76208d8ab729 100644 --- a/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java @@ -40,6 +40,7 @@ import io.druid.segment.loading.StorageLocationConfig; import io.druid.server.initialization.IndexerZkConfig; import io.druid.server.initialization.ZkPathsConfig; +import io.druid.server.metrics.NoopServiceEmitter; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.CuratorFrameworkFactory; import org.apache.curator.retry.ExponentialBackoffRetry; @@ -123,13 +124,14 @@ public String getBase() private WorkerTaskMonitor createTaskMonitor() { + final TaskConfig taskConfig = new TaskConfig(Files.createTempDir().toString(), null, null, 0, null, null, null); return new WorkerTaskMonitor( jsonMapper, cf, workerCuratorCoordinator, new ThreadPoolTaskRunner( new TaskToolboxFactory( - new TaskConfig(Files.createTempDir().toString(), null, null, 0, null), + taskConfig, null, null, null, null, null, null, null, null, null, null, null, new SegmentLoaderFactory( new SegmentLoaderLocalCacheManager( null, @@ -143,9 +145,13 @@ public List getLocations() } , jsonMapper ) - ), jsonMapper ), - null + jsonMapper, + null, + null + ), + taskConfig, + new NoopServiceEmitter() ), new WorkerConfig().setCapacity(1) ); diff --git a/pom.xml b/pom.xml index 92f1eaf7d95b..a66a3746d457 100644 --- a/pom.xml +++ b/pom.xml @@ -66,13 +66,13 @@ 0.27.4 - 2.8.0 + 2.9.1 9.2.5.v20141112 1.19 0.3.13 2.4.6 - 2.3 + 2.4.1 1.7.12 2.3.0 @@ -104,6 +104,8 @@ extensions/azure-extensions extensions/namespace-lookup extensions/kafka-extraction-namespace + extensions/datasketches + extensions-distribution distribution diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java index 6417bd3d891a..108e4c3ce034 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java @@ -30,7 +30,9 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Ordering; +import com.google.common.collect.Sets; import com.google.inject.Inject; +import com.metamx.common.IAE; import com.metamx.common.ISE; import com.metamx.common.Pair; import com.metamx.common.guava.Accumulator; @@ -160,14 +162,34 @@ private Sequence mergeGroupByResults( } final Sequence subqueryResult = mergeGroupByResults(subquery, runner, context); - final List aggs = Lists.newArrayList(); + final Set aggs = Sets.newHashSet(); + + // Nested group-bys work by first running the inner query and then materializing the results in an incremental + // index which the outer query is then run against. To build the incremental index, we use the fieldNames from + // the aggregators for the outer query to define the column names so that the index will match the query. If + // there are multiple types of aggregators in the outer query referencing the same fieldName, we will try to build + // multiple columns of the same name using different aggregator types and will fail. Here, we permit multiple + // aggregators of the same type referencing the same fieldName (and skip creating identical columns for the + // subsequent ones) and return an error if the aggregator types are different. for (AggregatorFactory aggregatorFactory : query.getAggregatorSpecs()) { - aggs.addAll(aggregatorFactory.getRequiredColumns()); + for (final AggregatorFactory transferAgg : aggregatorFactory.getRequiredColumns()) { + if (Iterables.any(aggs, new Predicate() { + @Override + public boolean apply(AggregatorFactory agg) { + return agg.getName().equals(transferAgg.getName()) && !agg.equals(transferAgg); + } + })) { + throw new IAE("Inner aggregator can currently only be referenced by a single type of outer aggregator" + + " for '%s'", transferAgg.getName()); + } + + aggs.add(transferAgg); + } } // We need the inner incremental index to have all the columns required by the outer query final GroupByQuery innerQuery = new GroupByQuery.Builder(subquery) - .setAggregatorSpecs(aggs) + .setAggregatorSpecs(Lists.newArrayList(aggs)) .setInterval(subquery.getIntervals()) .setPostAggregatorSpecs(Lists.newArrayList()) .build(); diff --git a/processing/src/main/java/io/druid/segment/IndexMerger.java b/processing/src/main/java/io/druid/segment/IndexMerger.java index 063cfbab644a..049cc7567a0b 100644 --- a/processing/src/main/java/io/druid/segment/IndexMerger.java +++ b/processing/src/main/java/io/druid/segment/IndexMerger.java @@ -58,6 +58,7 @@ import io.druid.guice.GuiceInjectors; import io.druid.guice.JsonConfigProvider; import io.druid.query.aggregation.AggregatorFactory; +import io.druid.segment.column.BitmapIndexSeeker; import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.ColumnCapabilitiesImpl; import io.druid.segment.column.ValueType; @@ -224,8 +225,11 @@ public static File mergeQueryableIndex( ProgressIndicator progress ) throws IOException { - return merge( - Lists.transform( + // We are materializing the list for performance reasons. Lists.transform + // only creates a "view" of the original list, meaning the function gets + // applied every time you access an element. + List indexAdapteres = Lists.newArrayList( + Iterables.transform( indexes, new Function() { @@ -235,7 +239,10 @@ public IndexableAdapter apply(final QueryableIndex input) return new QueryableIndexIndexableAdapter(input); } } - ), + ) + ); + return merge( + indexAdapteres, metricAggs, outDir, null, @@ -850,13 +857,17 @@ public Rowboat apply(@Nullable Rowboat input) tree = new RTree(2, new LinearGutmanSplitStrategy(0, 50, bitmapFactory), bitmapFactory); } + BitmapIndexSeeker[] bitmapIndexSeeker = new BitmapIndexSeeker[indexes.size()]; + for (int j = 0; j < indexes.size(); j++) { + bitmapIndexSeeker[j] = indexes.get(j).getBitmapIndexSeeker(dimension); + } for (String dimVal : IndexedIterable.create(dimVals)) { progress.progress(); List> convertedInverteds = Lists.newArrayListWithCapacity(indexes.size()); for (int j = 0; j < indexes.size(); ++j) { convertedInverteds.add( new ConvertingIndexedInts( - indexes.get(j).getBitmapIndex(dimension, dimVal), rowNumConversions.get(j) + bitmapIndexSeeker[j].seek(dimVal), rowNumConversions.get(j) ) ); } @@ -1006,6 +1017,7 @@ private static class DimValueConverter private int currIndex; private String lastVal = null; + private String currValue; DimValueConverter( Indexed dimSet @@ -1015,6 +1027,7 @@ private static class DimValueConverter conversionBuf = ByteBuffer.allocateDirect(dimSet.size() * Ints.BYTES).asIntBuffer(); currIndex = 0; + currValue = null; } public void convert(String value, int index) @@ -1028,7 +1041,9 @@ public void convert(String value, int index) } return; } - String currValue = dimSet.get(currIndex); + if (currValue == null) { + currValue = dimSet.get(currIndex); + } while (currValue == null) { conversionBuf.position(conversionBuf.position() + 1); @@ -1045,6 +1060,8 @@ public void convert(String value, int index) ++currIndex; if (currIndex == dimSet.size()) { lastVal = value; + } else { + currValue = dimSet.get(currIndex); } } else if (currValue.compareTo(value) < 0) { throw new ISE( diff --git a/processing/src/main/java/io/druid/segment/IndexableAdapter.java b/processing/src/main/java/io/druid/segment/IndexableAdapter.java index 3c09d7a31559..94f0fbe5700d 100644 --- a/processing/src/main/java/io/druid/segment/IndexableAdapter.java +++ b/processing/src/main/java/io/druid/segment/IndexableAdapter.java @@ -17,6 +17,7 @@ package io.druid.segment; +import io.druid.segment.column.BitmapIndexSeeker; import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.data.Indexed; import io.druid.segment.data.IndexedInts; @@ -41,6 +42,8 @@ public interface IndexableAdapter IndexedInts getBitmapIndex(String dimension, String value); + BitmapIndexSeeker getBitmapIndexSeeker(String dimension); + String getMetricType(String metric); ColumnCapabilities getCapabilities(String column); diff --git a/processing/src/main/java/io/druid/segment/QueryableIndexIndexableAdapter.java b/processing/src/main/java/io/druid/segment/QueryableIndexIndexableAdapter.java index 5d64edea45bb..a365850e6a8e 100644 --- a/processing/src/main/java/io/druid/segment/QueryableIndexIndexableAdapter.java +++ b/processing/src/main/java/io/druid/segment/QueryableIndexIndexableAdapter.java @@ -27,10 +27,12 @@ import com.metamx.common.guava.CloseQuietly; import com.metamx.common.logger.Logger; import io.druid.segment.column.BitmapIndex; +import io.druid.segment.column.BitmapIndexSeeker; import io.druid.segment.column.Column; import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.ComplexColumn; import io.druid.segment.column.DictionaryEncodedColumn; +import io.druid.segment.column.EmptyBitmapIndexSeeker; import io.druid.segment.column.GenericColumn; import io.druid.segment.column.IndexedFloatsGenericColumn; import io.druid.segment.column.IndexedLongsGenericColumn; @@ -38,6 +40,7 @@ import io.druid.segment.data.ArrayBasedIndexedInts; import io.druid.segment.data.BitmapCompressedIndexedInts; import io.druid.segment.data.EmptyIndexedInts; +import io.druid.segment.data.GenericIndexed; import io.druid.segment.data.Indexed; import io.druid.segment.data.IndexedInts; import io.druid.segment.data.IndexedIterable; @@ -331,4 +334,76 @@ public ColumnCapabilities getCapabilities(String column) { return input.getColumn(column).getCapabilities(); } + + @Override + public BitmapIndexSeeker getBitmapIndexSeeker(String dimension) + { + final Column column = input.getColumn(dimension); + + if (column == null) { + return new EmptyBitmapIndexSeeker(); + } + + final BitmapIndex bitmaps = column.getBitmapIndex(); + if (bitmaps == null) { + return new EmptyBitmapIndexSeeker(); + } + + final Indexed dimSet = getDimValueLookup(dimension); + + // BitmapIndexSeeker is the main performance boost comes from. + // In the previous version of index merge, during the creation of invert index, we do something like + // merge sort of multiply bitmap indexes. It simply iterator all the previous sorted values, + // and "binary find" the id in each bitmap indexes, which involves disk IO and is really slow. + // Suppose we have N (which is 100 in our test) small segments, each have M (which is 50000 in our case) rows. + // In high cardinality scenario, we will almost have N * M uniq values. So the complexity will be O(N * M * M * LOG(M)). + + // There are 2 properties we did not use during the merging: + // 1. We always travel the dimension values sequentially + // 2. One single dimension value is valid only in one index when cardinality is high enough + // So we introduced the BitmapIndexSeeker, which can only seek value sequentially and can never seek back. + // By using this and the help of "getDimValueLookup", we only need to translate all dimension value to its ID once, + // and the translation is done by self-increase of the integer. We only need to change the CACHED value once after + // previous value is hit, renew the value and increase the ID. The complexity now is O(N * M * LOG(M)). + return new BitmapIndexSeeker() + { + private int currIndex = 0; + private String currVal = null; + private String lastVal = null; + + @Override + public IndexedInts seek(String value) + { + if (dimSet == null || dimSet.size() == 0) { + return new EmptyIndexedInts(); + } + if (lastVal != null) { + if (GenericIndexed.STRING_STRATEGY.compare(value, lastVal) <= 0) { + throw new ISE("Value[%s] is less than the last value[%s] I have, cannot be.", + value, lastVal); + } + return new EmptyIndexedInts(); + } + if (currVal == null) { + currVal = dimSet.get(currIndex); + } + int compareResult = GenericIndexed.STRING_STRATEGY.compare(currVal, value); + if (compareResult == 0) { + IndexedInts ret = new BitmapCompressedIndexedInts(bitmaps.getBitmap(currIndex)); + ++currIndex; + if (currIndex == dimSet.size()) { + lastVal = value; + } else { + currVal = dimSet.get(currIndex); + } + return ret; + } else if (compareResult < 0) { + throw new ISE("Skipped currValue[%s], currIndex[%,d]; incoming value[%s]", + currVal, currIndex, value); + } else { + return new EmptyIndexedInts(); + } + } + }; + } } diff --git a/processing/src/main/java/io/druid/segment/RowboatFilteringIndexAdapter.java b/processing/src/main/java/io/druid/segment/RowboatFilteringIndexAdapter.java index bc4059a836f0..b5046724224f 100644 --- a/processing/src/main/java/io/druid/segment/RowboatFilteringIndexAdapter.java +++ b/processing/src/main/java/io/druid/segment/RowboatFilteringIndexAdapter.java @@ -21,6 +21,7 @@ import com.google.common.base.Predicate; import com.google.common.collect.Iterables; +import io.druid.segment.column.BitmapIndexSeeker; import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.data.Indexed; import io.druid.segment.data.IndexedInts; @@ -92,4 +93,10 @@ public ColumnCapabilities getCapabilities(String column) { return baseAdapter.getCapabilities(column); } + + @Override + public BitmapIndexSeeker getBitmapIndexSeeker(String dimension) + { + return baseAdapter.getBitmapIndexSeeker(dimension); + } } diff --git a/processing/src/main/java/io/druid/segment/column/BitmapIndexSeeker.java b/processing/src/main/java/io/druid/segment/column/BitmapIndexSeeker.java new file mode 100644 index 000000000000..6b369bce0808 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/column/BitmapIndexSeeker.java @@ -0,0 +1,28 @@ +/* + * Druid - a distributed column store. + * Copyright 2012 - 2015 Metamarkets Group Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.druid.segment.column; + +import io.druid.segment.data.IndexedInts; + +/** + * Only support access in order + */ +public interface BitmapIndexSeeker +{ + public IndexedInts seek(String value); +} diff --git a/processing/src/main/java/io/druid/segment/column/EmptyBitmapIndexSeeker.java b/processing/src/main/java/io/druid/segment/column/EmptyBitmapIndexSeeker.java new file mode 100644 index 000000000000..9756f9fae216 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/column/EmptyBitmapIndexSeeker.java @@ -0,0 +1,32 @@ +/* + * Druid - a distributed column store. + * Copyright 2012 - 2015 Metamarkets Group Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.druid.segment.column; + +import io.druid.segment.data.EmptyIndexedInts; +import io.druid.segment.data.IndexedInts; + +public class EmptyBitmapIndexSeeker implements BitmapIndexSeeker +{ + + @Override + public IndexedInts seek(String value) + { + return new EmptyIndexedInts(); + } + +} diff --git a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexAdapter.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexAdapter.java index d593a68f1dc1..2a6399de3f6a 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexAdapter.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexAdapter.java @@ -22,21 +22,29 @@ import com.google.common.base.Function; import com.google.common.collect.Maps; import com.metamx.collections.bitmap.BitmapFactory; +import com.metamx.collections.bitmap.ImmutableBitmap; import com.metamx.collections.bitmap.MutableBitmap; +import com.metamx.common.ISE; import com.metamx.common.guava.FunctionalIterable; import com.metamx.common.logger.Logger; + import io.druid.segment.IndexableAdapter; import io.druid.segment.Rowboat; +import io.druid.segment.column.BitmapIndexSeeker; import io.druid.segment.column.ColumnCapabilities; +import io.druid.segment.column.EmptyBitmapIndexSeeker; import io.druid.segment.data.EmptyIndexedInts; +import io.druid.segment.data.GenericIndexed; import io.druid.segment.data.Indexed; import io.druid.segment.data.IndexedInts; import io.druid.segment.data.IndexedIterable; import io.druid.segment.data.ListIndexed; + import org.joda.time.Interval; import org.roaringbitmap.IntIterator; import javax.annotation.Nullable; + import java.io.IOException; import java.util.Iterator; import java.util.Map; @@ -235,70 +243,109 @@ public IndexedInts getBitmapIndex(String dimension, String value) return new EmptyIndexedInts(); } - return new IndexedInts() + return new BitmapIndexedInts(bitmapIndex); + } + + @Override + public String getMetricType(String metric) + { + return index.getMetricType(metric); + } + + @Override + public ColumnCapabilities getCapabilities(String column) + { + return index.getCapabilities(column); + } + + @Override + public BitmapIndexSeeker getBitmapIndexSeeker(String dimension) + { + final Map dimInverted = invertedIndexes.get(dimension); + if (dimInverted == null) { + return new EmptyBitmapIndexSeeker(); + } + + return new BitmapIndexSeeker() { - @Override - public int size() - { - return bitmapIndex.size(); - } + private String lastVal = null; @Override - public int get(int index) + public IndexedInts seek(String value) { - throw new UnsupportedOperationException("This is really slow, so it's just not supported."); + if (value != null && GenericIndexed.STRING_STRATEGY.compare(value, lastVal) <= 0) { + throw new ISE("Value[%s] is less than the last value[%s] I have, cannot be.", + value, lastVal); + } + lastVal = value; + final MutableBitmap bitmapIndex = dimInverted.get(value); + if (bitmapIndex == null) { + return new EmptyIndexedInts(); + } + return new BitmapIndexedInts(bitmapIndex); } + }; + } - @Override - public Iterator iterator() - { - return new Iterator() - { - IntIterator baseIter = bitmapIndex.iterator(); + static class BitmapIndexedInts implements IndexedInts { - @Override - public boolean hasNext() - { - return baseIter.hasNext(); - } + private final MutableBitmap bitmapIndex; - @Override - public Integer next() - { - return baseIter.next(); - } + BitmapIndexedInts(MutableBitmap bitmapIndex) + { + this.bitmapIndex = bitmapIndex; + } - @Override - public void remove() - { - throw new UnsupportedOperationException(); - } - }; - } + @Override + public int size() + { + return bitmapIndex.size(); + } - @Override - public void fill(int index, int[] toFill) - { - throw new UnsupportedOperationException("fill not supported"); - } + @Override + public int get(int index) + { + // Slow for concise bitmaps, but is fast with roaring bitmaps, so it's just not supported. + throw new UnsupportedOperationException("Not supported."); + } - @Override - public void close() throws IOException + @Override + public Iterator iterator() + { + return new Iterator() { + final IntIterator baseIter = bitmapIndex.iterator(); - } - }; - } + @Override + public boolean hasNext() + { + return baseIter.hasNext(); + } - @Override - public String getMetricType(String metric) - { - return index.getMetricType(metric); - } + @Override + public Integer next() + { + return baseIter.next(); + } - @Override - public ColumnCapabilities getCapabilities(String column) - { - return index.getCapabilities(column); + @Override + public void remove() + { + throw new UnsupportedOperationException(); + } + }; + } + + @Override + public void fill(int index, int[] toFill) + { + throw new UnsupportedOperationException("fill not supported"); + } + + @Override + public void close() throws IOException + { + + } } } diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java index d52e35e2f6b1..dfb7cde7bd41 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java @@ -2502,6 +2502,61 @@ public void testDifferentGroupingSubquery() TestHelper.assertExpectedObjects(expectedResults, results, ""); } + @Test + public void testDifferentGroupingSubqueryMultipleAggregatorsOnSameField() + { + GroupByQuery subquery = GroupByQuery + .builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias"))) + .setAggregatorSpecs( + Arrays.asList( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory("idx", "index") + ) + ) + .setPostAggregatorSpecs( + Lists.newArrayList( + new ArithmeticPostAggregator( + "post_agg", + "+", + Lists.newArrayList( + new FieldAccessPostAggregator("idx", "idx"), + new FieldAccessPostAggregator("idx", "idx") + ) + ) + ) + ) + .setGranularity(QueryRunnerTestHelper.dayGran) + .build(); + + GroupByQuery query = GroupByQuery + .builder() + .setDataSource(subquery) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setAggregatorSpecs( + Arrays.asList( + new DoubleMaxAggregatorFactory("idx1", "idx"), + new DoubleMaxAggregatorFactory("idx2", "idx"), + new DoubleMaxAggregatorFactory("idx3", "post_agg"), + new DoubleMaxAggregatorFactory("idx4", "post_agg") + ) + ) + .setGranularity(QueryRunnerTestHelper.dayGran) + .build(); + + List expectedResults = Arrays.asList( + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "idx1", 2900.0, "idx2", 2900.0, + "idx3", 5800.0, "idx4", 5800.0), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "idx1", 2505.0, "idx2", 2505.0, + "idx3", 5010.0, "idx4", 5010.0) + ); + + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + TestHelper.assertExpectedObjects(expectedResults, results, ""); + } + @Test public void testDifferentGroupingSubqueryWithFilter() diff --git a/processing/src/test/java/io/druid/segment/QueryableIndexIndexableAdapterTest.java b/processing/src/test/java/io/druid/segment/QueryableIndexIndexableAdapterTest.java new file mode 100644 index 000000000000..aac05a1ef84b --- /dev/null +++ b/processing/src/test/java/io/druid/segment/QueryableIndexIndexableAdapterTest.java @@ -0,0 +1,95 @@ +/* +* Licensed to Metamarkets Group Inc. (Metamarkets) under one +* or more contributor license agreements. See the NOTICE file +* distributed with this work for additional information +* regarding copyright ownership. Metamarkets licenses this file +* to you under the Apache License, Version 2.0 (the +* "License"); you may not use this file except in compliance +* with the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, +* software distributed under the License is distributed on an +* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +* KIND, either express or implied. See the License for the +* specific language governing permissions and limitations +* under the License. +*/ + +package io.druid.segment; + +import java.io.File; + +import io.druid.segment.column.BitmapIndexSeeker; +import io.druid.segment.data.CompressedObjectStrategy; +import io.druid.segment.data.ConciseBitmapSerdeFactory; +import io.druid.segment.data.IncrementalIndexTest; +import io.druid.segment.data.IndexedInts; +import io.druid.segment.incremental.IncrementalIndex; +import io.druid.segment.incremental.IncrementalIndexAdapter; + +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import com.metamx.common.ISE; + +public class QueryableIndexIndexableAdapterTest { + private static final IndexSpec INDEX_SPEC = IndexMergerTest.makeIndexSpec( + new ConciseBitmapSerdeFactory(), + CompressedObjectStrategy.CompressionStrategy.LZ4, + CompressedObjectStrategy.CompressionStrategy.LZ4 + ); + + @Rule + public final TemporaryFolder temporaryFolder = new TemporaryFolder(); + @Rule + public final CloserRule closer = new CloserRule(false); + + @Test + public void testGetBitmapIndexSeeker() throws Exception + { + final long timestamp = System.currentTimeMillis(); + IncrementalIndex toPersist = IncrementalIndexTest.createIndex(false, null); + IncrementalIndexTest.populateIndex(timestamp, toPersist); + + final File tempDir = temporaryFolder.newFolder(); + final IndexableAdapter incrementalAdapter = new IncrementalIndexAdapter( + toPersist.getInterval(), + toPersist, + INDEX_SPEC.getBitmapSerdeFactory().getBitmapFactory() + ); + + QueryableIndex index = closer.closeLater( + IndexIO.loadIndex( + IndexMerger.persist( + toPersist, + tempDir, + null, + INDEX_SPEC + ) + ) + ); + + IndexableAdapter adapter = new QueryableIndexIndexableAdapter(index); + BitmapIndexSeeker bitmapIndexSeeker = adapter.getBitmapIndexSeeker("dim1"); + IndexedInts indexedInts0 = bitmapIndexSeeker.seek("0"); + Assert.assertEquals(0, indexedInts0.size()); + IndexedInts indexedInts1 = bitmapIndexSeeker.seek("1"); + Assert.assertEquals(1, indexedInts1.size()); + try { + bitmapIndexSeeker.seek("4"); + Assert.assertFalse("Only support access in order", true); + } catch(ISE ise) { + Assert.assertTrue("Only support access in order", true); + } + IndexedInts indexedInts2 = bitmapIndexSeeker.seek("2"); + Assert.assertEquals(0, indexedInts2.size()); + IndexedInts indexedInts3 = bitmapIndexSeeker.seek("3"); + Assert.assertEquals(1, indexedInts3.size()); + IndexedInts indexedInts4 = bitmapIndexSeeker.seek("4"); + Assert.assertEquals(0, indexedInts4.size()); + } +} diff --git a/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexAdapterTest.java b/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexAdapterTest.java new file mode 100644 index 000000000000..1f8b4eb1bcb5 --- /dev/null +++ b/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexAdapterTest.java @@ -0,0 +1,70 @@ +/* + * Druid - a distributed column store. + * Copyright 2012 - 2015 Metamarkets Group Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.druid.segment.incremental; + +import io.druid.segment.IndexSpec; +import io.druid.segment.IndexableAdapter; +import io.druid.segment.column.BitmapIndexSeeker; +import io.druid.segment.data.CompressedObjectStrategy; +import io.druid.segment.data.ConciseBitmapSerdeFactory; +import io.druid.segment.data.IncrementalIndexTest; +import io.druid.segment.data.IndexedInts; + +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Test; + +import com.metamx.common.ISE; + +public class IncrementalIndexAdapterTest { + private static final IndexSpec INDEX_SPEC = new IndexSpec( + new ConciseBitmapSerdeFactory(), + CompressedObjectStrategy.CompressionStrategy.LZ4.name().toLowerCase(), + CompressedObjectStrategy.CompressionStrategy.LZ4.name().toLowerCase() + ); + + @Test + public void testGetBitmapIndexSeeker() throws Exception + { + final long timestamp = System.currentTimeMillis(); + IncrementalIndex incrementalIndex = IncrementalIndexTest.createIndex(false, null); + IncrementalIndexTest.populateIndex(timestamp, incrementalIndex); + IndexableAdapter adapter = new IncrementalIndexAdapter( + incrementalIndex.getInterval(), + incrementalIndex, + INDEX_SPEC.getBitmapSerdeFactory().getBitmapFactory() + ); + BitmapIndexSeeker bitmapIndexSeeker = adapter.getBitmapIndexSeeker("dim1"); + IndexedInts indexedInts0 = bitmapIndexSeeker.seek("0"); + Assert.assertEquals(0, indexedInts0.size()); + IndexedInts indexedInts1 = bitmapIndexSeeker.seek("1"); + Assert.assertEquals(1, indexedInts1.size()); + try { + bitmapIndexSeeker.seek("01"); + Assert.assertFalse("Only support access in order", true); + } catch(ISE ise) { + Assert.assertTrue("Only support access in order", true); + } + IndexedInts indexedInts2 = bitmapIndexSeeker.seek("2"); + Assert.assertEquals(0, indexedInts2.size()); + IndexedInts indexedInts3 = bitmapIndexSeeker.seek("3"); + Assert.assertEquals(1, indexedInts3.size()); + IndexedInts indexedInts4 = bitmapIndexSeeker.seek("4"); + Assert.assertEquals(0, indexedInts4.size()); + } +} diff --git a/server/src/main/java/io/druid/curator/discovery/DiscoveryModule.java b/server/src/main/java/io/druid/curator/discovery/DiscoveryModule.java index 328d649140c4..11753bfbf475 100644 --- a/server/src/main/java/io/druid/curator/discovery/DiscoveryModule.java +++ b/server/src/main/java/io/druid/curator/discovery/DiscoveryModule.java @@ -37,6 +37,7 @@ import io.druid.server.DruidNode; import io.druid.server.initialization.CuratorDiscoveryConfig; import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.utils.CloseableExecutorService; import org.apache.curator.x.discovery.DownInstancePolicy; import org.apache.curator.x.discovery.InstanceFilter; import org.apache.curator.x.discovery.ProviderStrategy; @@ -55,6 +56,7 @@ import java.util.List; import java.util.Set; import java.util.concurrent.Executor; +import java.util.concurrent.ExecutorService; import java.util.concurrent.ThreadFactory; /** @@ -323,6 +325,18 @@ public ServiceCacheBuilder threadFactory(ThreadFactory threadFactory) return this; } + @Override + public ServiceCacheBuilder executorService(ExecutorService executorService) + { + return this; + } + + @Override + public ServiceCacheBuilder executorService(CloseableExecutorService closeableExecutorService) + { + return this; + } + private static class NoopServiceCache implements ServiceCache { @Override diff --git a/server/src/main/java/io/druid/curator/discovery/ServerDiscoverySelector.java b/server/src/main/java/io/druid/curator/discovery/ServerDiscoverySelector.java index c94cd6cb4221..a52a4a2cdd96 100644 --- a/server/src/main/java/io/druid/curator/discovery/ServerDiscoverySelector.java +++ b/server/src/main/java/io/druid/curator/discovery/ServerDiscoverySelector.java @@ -17,6 +17,8 @@ package io.druid.curator.discovery; +import com.google.common.base.Function; +import com.google.common.collect.Collections2; import com.google.common.net.HostAndPort; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; @@ -27,6 +29,8 @@ import org.apache.curator.x.discovery.ServiceProvider; import java.io.IOException; +import java.util.Collection; +import java.util.Collections; /** */ @@ -41,6 +45,40 @@ public ServerDiscoverySelector(ServiceProvider serviceProvider) this.serviceProvider = serviceProvider; } + private static final Function TO_SERVER = new Function() + { + @Override + public Server apply(final ServiceInstance instance) + { + return new Server() + { + @Override + public String getHost() + { + return HostAndPort.fromParts(getAddress(), getPort()).toString(); + } + + @Override + public String getAddress() + { + return instance.getAddress(); + } + + @Override + public int getPort() + { + return instance.getPort(); + } + + @Override + public String getScheme() + { + return "http"; + } + }; + } + }; + @Override public Server pick() { @@ -58,32 +96,18 @@ public Server pick() return null; } - return new Server() - { - @Override - public String getHost() - { - return HostAndPort.fromParts(getAddress(), getPort()).toString(); - } - - @Override - public String getAddress() - { - return instance.getAddress(); - } - - @Override - public int getPort() - { - return instance.getPort(); - } + return TO_SERVER.apply(instance); + } - @Override - public String getScheme() - { - return "http"; - } - }; + public Collection getAll() + { + try { + return Collections2.transform(serviceProvider.getAllInstances(), TO_SERVER); + } + catch (Exception e) { + log.info(e, "Unable to get all instances"); + return Collections.emptyList(); + } } @LifecycleStart diff --git a/server/src/main/java/io/druid/guice/annotations/RemoteChatHandler.java b/server/src/main/java/io/druid/guice/annotations/RemoteChatHandler.java new file mode 100644 index 000000000000..4b048c485b53 --- /dev/null +++ b/server/src/main/java/io/druid/guice/annotations/RemoteChatHandler.java @@ -0,0 +1,34 @@ +/* +* Licensed to Metamarkets Group Inc. (Metamarkets) under one +* or more contributor license agreements. See the NOTICE file +* distributed with this work for additional information +* regarding copyright ownership. Metamarkets licenses this file +* to you under the Apache License, Version 2.0 (the +* "License"); you may not use this file except in compliance +* with the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, +* software distributed under the License is distributed on an +* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +* KIND, either express or implied. See the License for the +* specific language governing permissions and limitations +* under the License. +*/ + +package io.druid.guice.annotations; + +import com.google.inject.BindingAnnotation; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +@BindingAnnotation +@Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD}) +@Retention(RetentionPolicy.RUNTIME) +public @interface RemoteChatHandler +{ +} diff --git a/server/src/main/java/io/druid/initialization/Initialization.java b/server/src/main/java/io/druid/initialization/Initialization.java index e98c2d99236f..80056793d51f 100644 --- a/server/src/main/java/io/druid/initialization/Initialization.java +++ b/server/src/main/java/io/druid/initialization/Initialization.java @@ -370,6 +370,7 @@ public static Injector makeInjectorWithModules(final Injector baseInjector, Iter { final ModuleList defaultModules = new ModuleList(baseInjector); defaultModules.addModules( + new Log4jShutterDownerModule(), new LifecycleModule(), EmitterModule.class, HttpClientModule.global(), diff --git a/server/src/main/java/io/druid/initialization/Log4jShutterDownerModule.java b/server/src/main/java/io/druid/initialization/Log4jShutterDownerModule.java new file mode 100644 index 000000000000..46210896037f --- /dev/null +++ b/server/src/main/java/io/druid/initialization/Log4jShutterDownerModule.java @@ -0,0 +1,115 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.initialization; + +import com.google.inject.Binder; +import com.google.inject.Key; +import com.google.inject.Module; +import com.google.inject.Provides; +import com.google.inject.name.Names; +import com.metamx.common.lifecycle.LifecycleStart; +import com.metamx.common.lifecycle.LifecycleStop; +import com.metamx.common.logger.Logger; +import io.druid.common.config.Log4jShutdown; +import io.druid.guice.ManageLifecycle; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.core.impl.Log4jContextFactory; +import org.apache.logging.log4j.core.util.ShutdownCallbackRegistry; +import org.apache.logging.log4j.spi.LoggerContextFactory; + +public class Log4jShutterDownerModule implements Module +{ + private static final Logger log = new Logger(Log4jShutterDownerModule.class); + + @Override + public void configure(Binder binder) + { + // Instantiate eagerly so that we get everything registered and put into the Lifecycle + // This makes the shutdown run pretty darn near last. + + try { + // Reflection to try and allow non Log4j2 stuff to run. This acts as a gateway to stop errors in the next few lines + final Class logManagerClazz = Class.forName("org.apache.logging.log4j.LogManager"); + + final LoggerContextFactory contextFactory = LogManager.getFactory(); + if (!(contextFactory instanceof Log4jContextFactory)) { + log.warn( + "Expected [%s] found [%s]. Unknown class for context factory. Not logging shutdown", + Log4jContextFactory.class.getCanonicalName(), + contextFactory.getClass().getCanonicalName() + ); + return; + } + final ShutdownCallbackRegistry registry = ((Log4jContextFactory) contextFactory).getShutdownCallbackRegistry(); + if (!(registry instanceof Log4jShutdown)) { + log.warn( + "Shutdown callback registry expected class [%s] found [%s]. Skipping shutdown registry", + Log4jShutdown.class.getCanonicalName(), + registry.getClass().getCanonicalName() + ); + return; + } + binder.bind(Log4jShutdown.class).toInstance((Log4jShutdown) registry); + binder.bind(Key.get(Log4jShutterDowner.class, Names.named("ForTheEagerness"))) + .to(Log4jShutterDowner.class) + .asEagerSingleton(); + } + catch (ClassNotFoundException | ClassCastException | LinkageError e) { + log.warn(e, "Not registering log4j shutdown hooks. Not using log4j?"); + } + } + + + @ManageLifecycle + @Provides + public Log4jShutterDowner getShutterDowner( + Log4jShutdown log4jShutdown + ) + { + return new Log4jShutterDowner(log4jShutdown); + } + + public static class Log4jShutterDowner + { + private final Log4jShutdown log4jShutdown; + + public Log4jShutterDowner(Log4jShutdown log4jShutdown) + { + this.log4jShutdown = log4jShutdown; + } + + @LifecycleStart + public void start() + { + log.debug("Log4j shutter downer is waiting"); + } + + @LifecycleStop + public void stop() + { + if (log4jShutdown != null) { + log.debug("Shutting down log4j"); + log4jShutdown.stop(); + } else { + log.warn("Log4j shutdown was registered in lifecycle but no shutdown object exists!"); + } + } + } +} diff --git a/server/src/main/java/io/druid/segment/indexing/DataSchema.java b/server/src/main/java/io/druid/segment/indexing/DataSchema.java index b628424b7f59..800c8cc6b4e9 100644 --- a/server/src/main/java/io/druid/segment/indexing/DataSchema.java +++ b/server/src/main/java/io/druid/segment/indexing/DataSchema.java @@ -101,6 +101,7 @@ public InputRowParser getParser() final Set dimensionExclusions = Sets.newHashSet(); for (AggregatorFactory aggregator : aggregators) { dimensionExclusions.addAll(aggregator.requiredFields()); + dimensionExclusions.add(aggregator.getName()); } if (inputRowParser.getParseSpec() != null) { diff --git a/server/src/main/java/io/druid/segment/realtime/firehose/EventReceiverFirehoseFactory.java b/server/src/main/java/io/druid/segment/realtime/firehose/EventReceiverFirehoseFactory.java index cb6c947dd03b..6f11ea6d4172 100644 --- a/server/src/main/java/io/druid/segment/realtime/firehose/EventReceiverFirehoseFactory.java +++ b/server/src/main/java/io/druid/segment/realtime/firehose/EventReceiverFirehoseFactory.java @@ -33,12 +33,12 @@ import io.druid.data.input.Firehose; import io.druid.data.input.FirehoseFactory; import io.druid.data.input.InputRow; -import io.druid.data.input.Rows; import io.druid.data.input.impl.MapInputRowParser; - import io.druid.guice.annotations.Json; import io.druid.guice.annotations.Smile; -import java.io.InputStream; +import io.druid.server.metrics.EventReceiverFirehoseMetric; +import io.druid.server.metrics.EventReceiverFirehoseRegister; + import javax.servlet.http.HttpServletRequest; import javax.ws.rs.Consumes; import javax.ws.rs.POST; @@ -48,6 +48,7 @@ import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; import java.io.IOException; +import java.io.InputStream; import java.util.Collection; import java.util.List; import java.util.Map; @@ -70,6 +71,7 @@ public class EventReceiverFirehoseFactory implements FirehoseFactory chatHandlerProvider; private final ObjectMapper jsonMapper; private final ObjectMapper smileMapper; + private final EventReceiverFirehoseRegister eventReceiverFirehoseRegister; @JsonCreator public EventReceiverFirehoseFactory( @@ -77,7 +79,8 @@ public EventReceiverFirehoseFactory( @JsonProperty("bufferSize") Integer bufferSize, @JacksonInject ChatHandlerProvider chatHandlerProvider, @JacksonInject @Json ObjectMapper jsonMapper, - @JacksonInject @Smile ObjectMapper smileMapper + @JacksonInject @Smile ObjectMapper smileMapper, + @JacksonInject EventReceiverFirehoseRegister eventReceiverFirehoseRegister ) { Preconditions.checkNotNull(serviceName, "serviceName"); @@ -87,13 +90,13 @@ public EventReceiverFirehoseFactory( this.chatHandlerProvider = Optional.fromNullable(chatHandlerProvider); this.jsonMapper = jsonMapper; this.smileMapper = smileMapper; + this.eventReceiverFirehoseRegister = eventReceiverFirehoseRegister; } @Override public Firehose connect(MapInputRowParser firehoseParser) throws IOException { log.info("Connecting firehose: %s", serviceName); - final EventReceiverFirehose firehose = new EventReceiverFirehose(firehoseParser); if (chatHandlerProvider.isPresent()) { @@ -106,6 +109,8 @@ public Firehose connect(MapInputRowParser firehoseParser) throws IOException log.info("No chathandler detected"); } + eventReceiverFirehoseRegister.register(serviceName, firehose); + return firehose; } @@ -121,7 +126,7 @@ public int getBufferSize() return bufferSize; } - public class EventReceiverFirehose implements ChatHandler, Firehose + public class EventReceiverFirehose implements ChatHandler, Firehose, EventReceiverFirehoseMetric { private final BlockingQueue buffer; private final MapInputRowParser parser; @@ -172,17 +177,7 @@ public Response addAll( } try { - for (final InputRow row : rows) { - boolean added = false; - while (!closed && !added) { - added = buffer.offer(row, 500, TimeUnit.MILLISECONDS); - } - - if (!added) { - throw new IllegalStateException("Cannot add events to closed firehose!"); - } - } - + addRows(rows); return Response.ok( objectMapper.writeValueAsString(ImmutableMap.of("eventCount", events.size())), contentType @@ -202,8 +197,11 @@ public boolean hasMore() { synchronized (readLock) { try { - while (!closed && nextRow == null) { + while (nextRow == null) { nextRow = buffer.poll(500, TimeUnit.MILLISECONDS); + if (closed) { + break; + } } } catch (InterruptedException e) { @@ -243,14 +241,44 @@ public void run() }; } + @Override + public int getCurrentBufferSize() + { + // ArrayBlockingQueue's implementation of size() is thread-safe, so we can use that + return buffer.size(); + } + + @Override + public int getCapacity() + { + return bufferSize; + } + @Override public void close() throws IOException { - log.info("Firehose closing."); - closed = true; + if (!closed) { + log.info("Firehose closing."); + closed = true; + eventReceiverFirehoseRegister.unregister(serviceName); + if (chatHandlerProvider.isPresent()) { + chatHandlerProvider.get().unregister(serviceName); + } + } + } + + // public for tests + public void addRows(Iterable rows) throws InterruptedException + { + for (final InputRow row : rows) { + boolean added = false; + while (!closed && !added) { + added = buffer.offer(row, 500, TimeUnit.MILLISECONDS); + } - if (chatHandlerProvider.isPresent()) { - chatHandlerProvider.get().unregister(serviceName); + if (!added) { + throw new IllegalStateException("Cannot add events to closed firehose!"); + } } } } diff --git a/server/src/main/java/io/druid/segment/realtime/firehose/ServiceAnnouncingChatHandlerProvider.java b/server/src/main/java/io/druid/segment/realtime/firehose/ServiceAnnouncingChatHandlerProvider.java index 1f4e973c59b7..92d2bb8be12c 100644 --- a/server/src/main/java/io/druid/segment/realtime/firehose/ServiceAnnouncingChatHandlerProvider.java +++ b/server/src/main/java/io/druid/segment/realtime/firehose/ServiceAnnouncingChatHandlerProvider.java @@ -23,8 +23,8 @@ import com.metamx.common.ISE; import com.metamx.common.logger.Logger; import io.druid.curator.discovery.ServiceAnnouncer; -import io.druid.guice.annotations.Self; import io.druid.server.DruidNode; +import io.druid.guice.annotations.RemoteChatHandler; import java.util.concurrent.ConcurrentMap; @@ -43,7 +43,7 @@ public class ServiceAnnouncingChatHandlerProvider implements ChatHandlerProvider @Inject public ServiceAnnouncingChatHandlerProvider( - @Self DruidNode node, + @RemoteChatHandler DruidNode node, ServiceAnnouncer serviceAnnouncer ) { diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumber.java b/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumber.java index 04c003500354..015d54741a02 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumber.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumber.java @@ -17,11 +17,14 @@ package io.druid.segment.realtime.plumber; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Lists; import com.metamx.common.Granularity; import com.metamx.common.concurrent.ScheduledExecutors; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; +import io.druid.client.cache.Cache; +import io.druid.client.cache.CacheConfig; import io.druid.common.guava.ThreadRenamingCallable; import io.druid.concurrent.Execs; import io.druid.query.QueryRunnerFactoryConglomerate; @@ -59,7 +62,10 @@ public FlushingPlumber( ServiceEmitter emitter, QueryRunnerFactoryConglomerate conglomerate, DataSegmentAnnouncer segmentAnnouncer, - ExecutorService queryExecutorService + ExecutorService queryExecutorService, + Cache cache, + CacheConfig cacheConfig, + ObjectMapper objectMapper ) { super( @@ -72,7 +78,10 @@ public FlushingPlumber( queryExecutorService, null, null, - null + null, + cache, + cacheConfig, + objectMapper ); this.flushDuration = flushDuration; diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumberSchool.java b/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumberSchool.java index 423781dae14b..9aac90c496d4 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumberSchool.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumberSchool.java @@ -20,8 +20,11 @@ import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; import com.metamx.emitter.service.ServiceEmitter; +import io.druid.client.cache.Cache; +import io.druid.client.cache.CacheConfig; import io.druid.guice.annotations.Processing; import io.druid.query.QueryRunnerFactoryConglomerate; import io.druid.segment.indexing.DataSchema; @@ -46,6 +49,9 @@ public class FlushingPlumberSchool extends RealtimePlumberSchool private final QueryRunnerFactoryConglomerate conglomerate; private final DataSegmentAnnouncer segmentAnnouncer; private final ExecutorService queryExecutorService; + private final Cache cache; + private final CacheConfig cacheConfig; + private final ObjectMapper objectMapper; @JsonCreator public FlushingPlumberSchool( @@ -53,7 +59,10 @@ public FlushingPlumberSchool( @JacksonInject ServiceEmitter emitter, @JacksonInject QueryRunnerFactoryConglomerate conglomerate, @JacksonInject DataSegmentAnnouncer segmentAnnouncer, - @JacksonInject @Processing ExecutorService queryExecutorService + @JacksonInject @Processing ExecutorService queryExecutorService, + @JacksonInject Cache cache, + @JacksonInject CacheConfig cacheConfig, + @JacksonInject ObjectMapper objectMapper ) { super( @@ -63,7 +72,10 @@ public FlushingPlumberSchool( segmentAnnouncer, null, null, - queryExecutorService + queryExecutorService, + cache, + cacheConfig, + objectMapper ); this.flushDuration = flushDuration == null ? defaultFlushDuration : flushDuration; @@ -71,6 +83,9 @@ public FlushingPlumberSchool( this.conglomerate = conglomerate; this.segmentAnnouncer = segmentAnnouncer; this.queryExecutorService = queryExecutorService; + this.cache = cache; + this.cacheConfig = cacheConfig; + this.objectMapper = objectMapper; } @Override @@ -90,7 +105,10 @@ public Plumber findPlumber( emitter, conglomerate, segmentAnnouncer, - queryExecutorService + queryExecutorService, + cache, + cacheConfig, + objectMapper ); } diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java index 00a89969e6f1..e5ba74fb938d 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java @@ -19,6 +19,7 @@ package io.druid.segment.realtime.plumber; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.base.Joiner; import com.google.common.base.Predicate; @@ -39,14 +40,18 @@ import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; +import io.druid.client.CachingQueryRunner; import io.druid.client.FilteredServerView; import io.druid.client.ServerView; +import io.druid.client.cache.Cache; +import io.druid.client.cache.CacheConfig; import io.druid.common.guava.ThreadRenamingCallable; import io.druid.common.guava.ThreadRenamingRunnable; import io.druid.concurrent.Execs; import io.druid.data.input.Committer; import io.druid.data.input.InputRow; import io.druid.query.MetricsEmittingQueryRunner; +import io.druid.query.NoopQueryRunner; import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; @@ -62,6 +67,7 @@ import io.druid.segment.IndexSpec; import io.druid.segment.QueryableIndex; import io.druid.segment.QueryableIndexSegment; +import io.druid.segment.ReferenceCountingSegment; import io.druid.segment.Segment; import io.druid.segment.incremental.IndexSizeExceededException; import io.druid.segment.indexing.DataSchema; @@ -119,6 +125,10 @@ public class RealtimePlumber implements Plumber String.CASE_INSENSITIVE_ORDER ); + private final Cache cache; + private final CacheConfig cacheConfig; + private final ObjectMapper objectMapper; + private volatile long nextFlush = 0; private volatile boolean shuttingDown = false; private volatile boolean stopped = false; @@ -129,6 +139,8 @@ public class RealtimePlumber implements Plumber private static final String COMMIT_METADATA_KEY = "%commitMetadata%"; private static final String COMMIT_METADATA_TIMESTAMP_KEY = "%commitMetadataTimestamp%"; + private static final String SKIP_INCREMENTAL_SEGMENT = "skipIncrementalSegment"; + public RealtimePlumber( DataSchema schema, @@ -140,7 +152,10 @@ public RealtimePlumber( ExecutorService queryExecutorService, DataSegmentPusher dataSegmentPusher, SegmentPublisher segmentPublisher, - FilteredServerView serverView + FilteredServerView serverView, + Cache cache, + CacheConfig cacheConfig, + ObjectMapper objectMapper ) { this.schema = schema; @@ -154,6 +169,13 @@ public RealtimePlumber( this.dataSegmentPusher = dataSegmentPusher; this.segmentPublisher = segmentPublisher; this.serverView = serverView; + this.cache = cache; + this.cacheConfig = cacheConfig; + this.objectMapper = objectMapper; + + if(!cache.isLocal()) { + log.error("Configured cache is not local, caching will not be enabled"); + } log.info("Creating plumber using rejectionPolicy[%s]", getRejectionPolicy()); } @@ -248,6 +270,7 @@ private Sink getSink(long timestamp) @Override public QueryRunner getQueryRunner(final Query query) { + final boolean skipIncrementalSegment = query.getContextValue(SKIP_INCREMENTAL_SEGMENT, false); final QueryRunnerFactory> factory = conglomerate.findFactory(query); final QueryToolChest> toolchest = factory.getToolchest(); @@ -284,6 +307,7 @@ public QueryRunner apply(TimelineObjectHolder holder) // The realtime plumber always uses SingleElementPartitionChunk final Sink theSink = holder.getObject().getChunk(0).getObject(); + final boolean skipIncrementalSegment = query.getContextValue(SKIP_INCREMENTAL_SEGMENT, false); if (theSink == null) { throw new ISE("Missing sink for timeline entry[%s]!", holder); @@ -315,10 +339,30 @@ public QueryRunner apply(FireHydrant input) return new ReportTimelineMissingSegmentQueryRunner(descriptor); } + if (skipIncrementalSegment && !input.hasSwapped()) { + return new NoopQueryRunner(); + } + // Prevent the underlying segment from closing when its being iterated - final Closeable closeable = input.getSegment().increment(); + final ReferenceCountingSegment segment = input.getSegment(); + final Closeable closeable = segment.increment(); try { - return factory.createRunner(input.getSegment()); + if (input.hasSwapped() // only use caching if data is immutable + && cache.isLocal() // hydrants may not be in sync between replicas, make sure cache is local + ) { + return new CachingQueryRunner<>( + makeHydrantIdentifier(input, segment), + descriptor, + objectMapper, + cache, + toolchest, + factory.createRunner(segment), + MoreExecutors.sameThreadExecutor(), + cacheConfig + ); + } else { + return factory.createRunner(input.getSegment()); + } } finally { try { @@ -346,6 +390,11 @@ public QueryRunner apply(FireHydrant input) ); } + protected static String makeHydrantIdentifier(FireHydrant input, ReferenceCountingSegment segment) + { + return segment.getIdentifier() + "_" + input.getCount(); + } + @Override public void persist(final Committer committer) { @@ -894,6 +943,9 @@ protected void abandonSegment(final long truncatedTime, final Sink sink) sink.getVersion(), new SingleElementPartitionChunk<>(sink) ); + for (FireHydrant hydrant : sink) { + cache.close(makeHydrantIdentifier(hydrant, hydrant.getSegment())); + } synchronized (handoffCondition) { handoffCondition.notifyAll(); } diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchool.java b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchool.java index aac7e884c58f..68113727929d 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchool.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchool.java @@ -19,9 +19,12 @@ import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; import com.metamx.emitter.service.ServiceEmitter; import io.druid.client.FilteredServerView; +import io.druid.client.cache.Cache; +import io.druid.client.cache.CacheConfig; import io.druid.guice.annotations.Processing; import io.druid.query.QueryRunnerFactoryConglomerate; import io.druid.segment.indexing.DataSchema; @@ -44,6 +47,9 @@ public class RealtimePlumberSchool implements PlumberSchool private final SegmentPublisher segmentPublisher; private final FilteredServerView serverView; private final ExecutorService queryExecutorService; + private final Cache cache; + private final CacheConfig cacheConfig; + private final ObjectMapper objectMapper; @JsonCreator public RealtimePlumberSchool( @@ -53,8 +59,11 @@ public RealtimePlumberSchool( @JacksonInject DataSegmentAnnouncer segmentAnnouncer, @JacksonInject SegmentPublisher segmentPublisher, @JacksonInject FilteredServerView serverView, - @JacksonInject @Processing ExecutorService executorService - ) + @JacksonInject @Processing ExecutorService executorService, + @JacksonInject Cache cache, + @JacksonInject CacheConfig cacheConfig, + @JacksonInject ObjectMapper objectMapper + ) { this.emitter = emitter; this.conglomerate = conglomerate; @@ -63,6 +72,10 @@ public RealtimePlumberSchool( this.segmentPublisher = segmentPublisher; this.serverView = serverView; this.queryExecutorService = executorService; + + this.cache = cache; + this.cacheConfig = cacheConfig; + this.objectMapper = objectMapper; } @Override @@ -84,7 +97,10 @@ public Plumber findPlumber( queryExecutorService, dataSegmentPusher, segmentPublisher, - serverView + serverView, + cache, + cacheConfig, + objectMapper ); } diff --git a/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java b/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java index 0d1774cf7535..d89cefd83c4d 100644 --- a/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java +++ b/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java @@ -47,7 +47,10 @@ import javax.servlet.http.HttpServletResponse; import javax.ws.rs.core.MediaType; import java.io.IOException; +import java.io.UnsupportedEncodingException; import java.net.URI; +import java.net.URISyntaxException; +import java.net.URLDecoder; import java.util.UUID; import java.util.concurrent.TimeUnit; @@ -110,18 +113,46 @@ public AsyncQueryForwardingServlet( @Override protected void service(HttpServletRequest request, HttpServletResponse response) throws ServletException, IOException { - final boolean isSmile = SmileMediaTypes.APPLICATION_JACKSON_SMILE.equals(request.getContentType()) || APPLICATION_SMILE.equals(request.getContentType()); + final boolean isSmile = SmileMediaTypes.APPLICATION_JACKSON_SMILE.equals(request.getContentType()) + || APPLICATION_SMILE.equals(request.getContentType()); final ObjectMapper objectMapper = isSmile ? smileMapper : jsonMapper; request.setAttribute(OBJECTMAPPER_ATTRIBUTE, objectMapper); - String host = hostFinder.getDefaultHost(); - request.setAttribute(HOST_ATTRIBUTE, host); - - boolean isQuery = request.getMethod().equals(HttpMethod.POST.asString()) && - request.getRequestURI().startsWith("/druid/v2"); - - // queries only exist for POST - if (isQuery) { + final String defaultHost = hostFinder.getDefaultHost(); + request.setAttribute(HOST_ATTRIBUTE, defaultHost); + + final boolean isQueryEndpoint = request.getRequestURI().startsWith("/druid/v2"); + + if (isQueryEndpoint && HttpMethod.DELETE.is(request.getMethod())) { + // query cancellation request + for (final String host : hostFinder.getAllHosts()) { + // send query cancellation to all brokers this query may have gone to + // to keep the code simple, the proxy servlet will also send a request to one of the default brokers + if (!host.equals(defaultHost)) { + // issue async requests + getHttpClient() + .newRequest(rewriteURI(request, host)) + .method(HttpMethod.DELETE) + .send( + new Response.CompleteListener() + { + @Override + public void onComplete(Result result) + { + if (result.isFailed()) { + log.warn( + result.getFailure(), + "Failed to forward cancellation request to [%s]", + host + ); + } + } + } + ); + } + } + } else if (isQueryEndpoint && HttpMethod.POST.is(request.getMethod())) { + // query request try { Query inputQuery = objectMapper.readValue(request.getInputStream(), Query.class); if (inputQuery != null) { @@ -172,7 +203,8 @@ protected void customizeProxyRequest(Request proxyRequest, HttpServletRequest re final ObjectMapper objectMapper = (ObjectMapper) request.getAttribute(OBJECTMAPPER_ATTRIBUTE); try { proxyRequest.content(new BytesContentProvider(objectMapper.writeValueAsBytes(query))); - } catch(JsonProcessingException e) { + } + catch (JsonProcessingException e) { Throwables.propagate(e); } } @@ -194,16 +226,29 @@ protected Response.Listener newProxyResponseListener( @Override protected URI rewriteURI(HttpServletRequest request) { - final String host = (String) request.getAttribute(HOST_ATTRIBUTE); - final StringBuilder uri = new StringBuilder("http://"); - - uri.append(host); - uri.append(request.getRequestURI()); - final String queryString = request.getQueryString(); - if (queryString != null) { - uri.append("?").append(queryString); + return rewriteURI(request, (String) request.getAttribute(HOST_ATTRIBUTE)); + } + + protected URI rewriteURI(HttpServletRequest request, String host) + { + return makeURI(host, request.getRequestURI(), request.getQueryString()); + } + + protected static URI makeURI(String host, String requestURI, String rawQueryString) + { + try { + return new URI( + "http", + host, + requestURI, + rawQueryString == null ? null : URLDecoder.decode(rawQueryString, "UTF-8"), + null + ); + } + catch (UnsupportedEncodingException | URISyntaxException e) { + log.error(e, "Unable to rewrite URI [%s]", e.getMessage()); + throw Throwables.propagate(e); } - return URI.create(uri.toString()); } @Override @@ -261,7 +306,7 @@ public void onComplete(Result result) try { emitter.emit( DruidMetrics.makeQueryTimeMetric(jsonMapper, query, req.getRemoteAddr()) - .build("query/time", requestTime) + .build("query/time", requestTime) ); requestLogger.log( diff --git a/server/src/main/java/io/druid/server/initialization/jetty/ChatHandlerServerModule.java b/server/src/main/java/io/druid/server/initialization/jetty/ChatHandlerServerModule.java new file mode 100644 index 000000000000..8a42f8068737 --- /dev/null +++ b/server/src/main/java/io/druid/server/initialization/jetty/ChatHandlerServerModule.java @@ -0,0 +1,78 @@ +/* +* Licensed to Metamarkets Group Inc. (Metamarkets) under one +* or more contributor license agreements. See the NOTICE file +* distributed with this work for additional information +* regarding copyright ownership. Metamarkets licenses this file +* to you under the Apache License, Version 2.0 (the +* "License"); you may not use this file except in compliance +* with the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, +* software distributed under the License is distributed on an +* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +* KIND, either express or implied. See the License for the +* specific language governing permissions and limitations +* under the License. +*/ + +package io.druid.server.initialization.jetty; + +import com.google.inject.Binder; +import com.google.inject.Inject; +import com.google.inject.Injector; +import com.google.inject.Key; +import com.google.inject.Module; +import com.google.inject.Provides; +import com.metamx.common.lifecycle.Lifecycle; +import com.metamx.common.logger.Logger; +import io.druid.guice.JsonConfigProvider; +import io.druid.guice.LazySingleton; +import io.druid.guice.LifecycleModule; +import io.druid.guice.annotations.RemoteChatHandler; +import io.druid.guice.annotations.Self; +import io.druid.server.DruidNode; +import io.druid.server.initialization.ServerConfig; +import org.eclipse.jetty.server.Server; + +import java.util.Properties; + +/** + */ +public class ChatHandlerServerModule implements Module +{ + private static final Logger log = new Logger(ChatHandlerServerModule.class); + + @Inject + private Properties properties; + + @Override + public void configure(Binder binder) + { + /** If "druid.indexer.task.chathandler.port" property is set then we assume that a + * separate Jetty Server with it's own {@link ServerConfig} is required for ingestion apart from the query server + * otherwise we bind {@link DruidNode} annotated with {@link RemoteChatHandler} to {@literal @}{@link Self} {@link DruidNode} + * so that same Jetty Server is used for querying as well as ingestion + */ + if (properties.containsKey("druid.indexer.task.chathandler.port")) { + log.info("Spawning separate ingestion server at port [%s]", properties.get("druid.indexer.task.chathandler.port")); + JsonConfigProvider.bind(binder, "druid.indexer.task.chathandler", DruidNode.class, RemoteChatHandler.class); + JsonConfigProvider.bind(binder, "druid.indexer.server.chathandler.http", ServerConfig.class, RemoteChatHandler.class); + LifecycleModule.register(binder, Server.class, RemoteChatHandler.class); + } else { + binder.bind(DruidNode.class).annotatedWith(RemoteChatHandler.class).to(Key.get(DruidNode.class, Self.class)); + binder.bind(ServerConfig.class).annotatedWith(RemoteChatHandler.class).to(Key.get(ServerConfig.class)); + } + } + + @Provides + @LazySingleton + @RemoteChatHandler + public Server getServer(Injector injector, Lifecycle lifecycle, @RemoteChatHandler DruidNode node, @RemoteChatHandler ServerConfig config) + { + final Server server = JettyServerModule.makeJettyServer(node, config); + JettyServerModule.initializeServer(injector, lifecycle, server); + return server; + } +} diff --git a/server/src/main/java/io/druid/server/initialization/jetty/JettyServerModule.java b/server/src/main/java/io/druid/server/initialization/jetty/JettyServerModule.java index 45bd177251b0..775c1e5e952e 100644 --- a/server/src/main/java/io/druid/server/initialization/jetty/JettyServerModule.java +++ b/server/src/main/java/io/druid/server/initialization/jetty/JettyServerModule.java @@ -109,38 +109,8 @@ protected ResourceConfig getDefaultResourceConfig( @LazySingleton public Server getServer(Injector injector, Lifecycle lifecycle, @Self DruidNode node, ServerConfig config) { - JettyServerInitializer initializer = injector.getInstance(JettyServerInitializer.class); - final Server server = makeJettyServer(node, config); - try { - initializer.initialize(server, injector); - } - catch (ConfigurationException e) { - throw new ProvisionException(Iterables.getFirst(e.getErrorMessages(), null).getMessage()); - } - - - lifecycle.addHandler( - new Lifecycle.Handler() - { - @Override - public void start() throws Exception - { - server.start(); - } - - @Override - public void stop() - { - try { - server.stop(); - } - catch (Exception e) { - log.warn(e, "Unable to stop Jetty server."); - } - } - } - ); + initializeServer(injector, lifecycle, server); return server; } @@ -153,7 +123,7 @@ public JacksonJsonProvider getJacksonJsonProvider(@Json ObjectMapper objectMappe return provider; } - private static Server makeJettyServer(@Self DruidNode node, ServerConfig config) + static Server makeJettyServer(DruidNode node, ServerConfig config) { final QueuedThreadPool threadPool = new QueuedThreadPool(); threadPool.setMinThreads(config.getNumThreads()); @@ -177,4 +147,38 @@ private static Server makeJettyServer(@Self DruidNode node, ServerConfig config) return server; } + + static void initializeServer(Injector injector, Lifecycle lifecycle, final Server server) + { + JettyServerInitializer initializer = injector.getInstance(JettyServerInitializer.class); + try { + initializer.initialize(server, injector); + } + catch (ConfigurationException e) { + throw new ProvisionException(Iterables.getFirst(e.getErrorMessages(), null).getMessage()); + } + + lifecycle.addHandler( + new Lifecycle.Handler() + { + @Override + public void start() throws Exception + { + server.start(); + } + + @Override + public void stop() + { + try { + server.stop(); + } + catch (Exception e) { + log.warn(e, "Unable to stop Jetty server."); + } + } + } + ); + } + } diff --git a/server/src/main/java/io/druid/server/metrics/EventReceiverFirehoseMetric.java b/server/src/main/java/io/druid/server/metrics/EventReceiverFirehoseMetric.java new file mode 100644 index 000000000000..8f286d05c588 --- /dev/null +++ b/server/src/main/java/io/druid/server/metrics/EventReceiverFirehoseMetric.java @@ -0,0 +1,41 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.server.metrics; + +/** + * An EventReceiverFirehoseMetric is an object with metrics about EventReceiverFirehose objects. + * It is not likely that anything other than an EventReceiverFirehose actually implements this. + * This interface is not part of the public API and backwards incompatible changes can occur without + * requiring a major (or even minor) version change. + * The interface's primary purpose is to be able to share metrics via the EventReceiverFirehoseRegister + * without exposing the entire EventReceiverFirehose + */ +public interface EventReceiverFirehoseMetric +{ + /** + * Return the current number of {@link io.druid.data.input.InputRow} that are stored in the buffer. + */ + int getCurrentBufferSize(); + + /** + * Return the capacity of the buffer. + */ + int getCapacity(); +} diff --git a/server/src/main/java/io/druid/server/metrics/EventReceiverFirehoseMonitor.java b/server/src/main/java/io/druid/server/metrics/EventReceiverFirehoseMonitor.java new file mode 100644 index 000000000000..aa11bf589d61 --- /dev/null +++ b/server/src/main/java/io/druid/server/metrics/EventReceiverFirehoseMonitor.java @@ -0,0 +1,61 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.server.metrics; + +import com.google.inject.Inject; +import com.metamx.emitter.service.ServiceEmitter; +import com.metamx.emitter.service.ServiceMetricEvent; +import com.metamx.metrics.AbstractMonitor; + +import java.util.Map; + +public class EventReceiverFirehoseMonitor extends AbstractMonitor +{ + + private final EventReceiverFirehoseRegister register; + + @Inject + public EventReceiverFirehoseMonitor( + EventReceiverFirehoseRegister eventReceiverFirehoseRegister + ) + { + this.register = eventReceiverFirehoseRegister; + } + + @Override + public boolean doMonitor(ServiceEmitter emitter) + { + for (Map.Entry entry : register.getMetrics()) { + final String serviceName = entry.getKey(); + final EventReceiverFirehoseMetric metric = entry.getValue(); + + final ServiceMetricEvent.Builder builder = ServiceMetricEvent.builder() + .setDimension("serviceName", serviceName) + .setDimension( + "bufferCapacity", + String.valueOf(metric.getCapacity()) + ); + + emitter.emit(builder.build("ingest/events/buffered", metric.getCurrentBufferSize())); + } + + return true; + } +} diff --git a/server/src/main/java/io/druid/server/metrics/EventReceiverFirehoseRegister.java b/server/src/main/java/io/druid/server/metrics/EventReceiverFirehoseRegister.java new file mode 100644 index 000000000000..da1f073e461c --- /dev/null +++ b/server/src/main/java/io/druid/server/metrics/EventReceiverFirehoseRegister.java @@ -0,0 +1,56 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.server.metrics; + +import com.metamx.common.ISE; +import com.metamx.common.logger.Logger; + +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; + +public class EventReceiverFirehoseRegister +{ + + private static final Logger log = new Logger(EventReceiverFirehoseRegister.class); + + private final ConcurrentMap metrics = new ConcurrentHashMap<>(); + + public void register(String serviceName, EventReceiverFirehoseMetric metric) + { + log.info("Registering EventReceiverFirehoseMetric for service [%s]", serviceName); + if (metrics.putIfAbsent(serviceName, metric) != null) { + throw new ISE("Service [%s] is already registered!", serviceName); + } + } + + public Iterable> getMetrics() + { + return metrics.entrySet(); + } + + public void unregister(String serviceName) + { + log.info("Unregistering EventReceiverFirehoseMetric for service [%s]", serviceName); + if (metrics.remove(serviceName) == null) { + log.warn("Unregistering a non-exist service. Service [%s] never exists."); + } + } +} diff --git a/server/src/main/java/io/druid/server/metrics/MetricsModule.java b/server/src/main/java/io/druid/server/metrics/MetricsModule.java index a395b8b317eb..2d80a760d2e3 100644 --- a/server/src/main/java/io/druid/server/metrics/MetricsModule.java +++ b/server/src/main/java/io/druid/server/metrics/MetricsModule.java @@ -33,6 +33,7 @@ import io.druid.concurrent.Execs; import io.druid.guice.DruidBinders; import io.druid.guice.JsonConfigProvider; +import io.druid.guice.LazySingleton; import io.druid.guice.ManageLifecycle; import java.util.List; @@ -59,6 +60,8 @@ public void configure(Binder binder) DruidBinders.metricMonitorBinder(binder); // get the binder so that it will inject the empty set at a minimum. + binder.bind(EventReceiverFirehoseRegister.class).in(LazySingleton.class); + // Instantiate eagerly so that we get everything registered and put into the Lifecycle binder.bind(Key.get(MonitorScheduler.class, Names.named("ForTheEagerness"))) .to(MonitorScheduler.class) diff --git a/server/src/main/java/io/druid/server/router/QueryHostFinder.java b/server/src/main/java/io/druid/server/router/QueryHostFinder.java index 9c5e854bffb6..cacbb76a7e34 100644 --- a/server/src/main/java/io/druid/server/router/QueryHostFinder.java +++ b/server/src/main/java/io/druid/server/router/QueryHostFinder.java @@ -17,6 +17,8 @@ package io.druid.server.router; +import com.google.common.base.Function; +import com.google.common.collect.FluentIterable; import com.google.inject.Inject; import com.metamx.common.ISE; import com.metamx.common.Pair; @@ -25,17 +27,18 @@ import io.druid.curator.discovery.ServerDiscoverySelector; import io.druid.query.Query; +import java.util.Collection; import java.util.concurrent.ConcurrentHashMap; /** */ -public class QueryHostFinder +public class QueryHostFinder { private static EmittingLogger log = new EmittingLogger(QueryHostFinder.class); private final TieredBrokerHostSelector hostSelector; - private final ConcurrentHashMap serverBackup = new ConcurrentHashMap(); + private final ConcurrentHashMap serverBackup = new ConcurrentHashMap<>(); @Inject public QueryHostFinder( @@ -45,7 +48,7 @@ public QueryHostFinder( this.hostSelector = hostSelector; } - public Server findServer(Query query) + public Server findServer(Query query) { final Pair selected = hostSelector.select(query); return findServerInner(selected); @@ -57,7 +60,30 @@ public Server findDefaultServer() return findServerInner(selected); } - public String getHost(Query query) + public Collection getAllHosts() + { + return FluentIterable + .from((Collection) hostSelector.getAllBrokers().values()) + .transformAndConcat( + new Function>() + { + @Override + public Iterable apply(ServerDiscoverySelector input) + { + return input.getAll(); + } + } + ).transform(new Function() + { + @Override + public String apply(Server input) + { + return input.getHost(); + } + }).toList(); + } + + public String getHost(Query query) { Server server = findServer(query); @@ -69,9 +95,10 @@ public String getHost(Query query) throw new ISE("No server found for query[%s]", query); } - log.debug("Selected [%s]", server.getHost()); + final String host = server.getHost(); + log.debug("Selected [%s]", host); - return server.getHost(); + return host; } public String getDefaultHost() diff --git a/server/src/main/java/io/druid/server/router/TieredBrokerHostSelector.java b/server/src/main/java/io/druid/server/router/TieredBrokerHostSelector.java index f061b5c53ae5..e8ceb90ace05 100644 --- a/server/src/main/java/io/druid/server/router/TieredBrokerHostSelector.java +++ b/server/src/main/java/io/druid/server/router/TieredBrokerHostSelector.java @@ -34,6 +34,7 @@ import org.joda.time.DateTime; import org.joda.time.Interval; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; @@ -201,4 +202,9 @@ public Pair getDefaultLookup() final ServerDiscoverySelector retVal = selectorMap.get(brokerServiceName); return new Pair<>(brokerServiceName, retVal); } + + public Map getAllBrokers() + { + return Collections.unmodifiableMap(selectorMap); + } } diff --git a/server/src/test/java/io/druid/curator/discovery/ServiceAnnouncerTest.java b/server/src/test/java/io/druid/curator/discovery/ServiceAnnouncerTest.java new file mode 100644 index 000000000000..bce613e0157b --- /dev/null +++ b/server/src/test/java/io/druid/curator/discovery/ServiceAnnouncerTest.java @@ -0,0 +1,108 @@ +/* + * Druid - a distributed column store. + * Copyright 2012 - 2015 Metamarkets Group Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.druid.curator.discovery; + +import com.google.common.base.Predicate; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterators; +import com.metamx.common.ISE; +import io.druid.curator.CuratorTestBase; +import org.apache.curator.x.discovery.ServiceDiscovery; +import org.apache.curator.x.discovery.ServiceDiscoveryBuilder; +import org.apache.curator.x.discovery.ServiceInstance; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.List; + +public class ServiceAnnouncerTest extends CuratorTestBase +{ + @Before + public void setUp() throws Exception + { + setupServerAndCurator(); + } + + @Test + public void testServiceAnnouncement() throws Exception + { + curator.start(); + List serviceNames = ImmutableList.of( + "druid/overlord", + "druid/coordinator", + "druid/firehose/tranquility_test-50-0000-0000" + ); + final ServiceDiscovery serviceDiscovery = createAndAnnounceServices(serviceNames); + Assert.assertTrue( + Iterators.all( + serviceNames.iterator(), + new Predicate() + { + @Override + public boolean apply(String input) + { + try { + return serviceDiscovery.queryForInstances(input.replaceAll("/", ":")).size() == 1; + } + catch (Exception e) { + throw new ISE( + "Something went wrong while finding instance with name [%s] in Service Discovery", + input + ); + } + } + } + ) + ); + } + + @Test (expected = IllegalArgumentException.class) + public void testServiceAnnouncementFail() throws Exception + { + curator.start(); + createAndAnnounceServices(ImmutableList.of("placeholder/\u0001")); + } + + private ServiceDiscovery createAndAnnounceServices(List serviceNames) throws Exception + { + int port = 1000; + ServiceDiscovery serviceDiscovery = + ServiceDiscoveryBuilder.builder(Void.class) + .basePath("/test") + .client(curator) + .build(); + for (String serviceName: serviceNames) { + String serviceNameToUse = CuratorServiceUtils.makeCanonicalServiceName(serviceName); + ServiceInstance instance = ServiceInstance.builder() + .name(serviceNameToUse) + .address("localhost") + .port(port++) + .build(); + serviceDiscovery.registerService(instance); + } + return serviceDiscovery; + } + + @After + public void tearDown() + { + tearDownServerAndCurator(); + } +} diff --git a/server/src/test/java/io/druid/segment/indexing/DataSchemaTest.java b/server/src/test/java/io/druid/segment/indexing/DataSchemaTest.java index 9059a7928d9c..4b2d534d4de4 100644 --- a/server/src/test/java/io/druid/segment/indexing/DataSchemaTest.java +++ b/server/src/test/java/io/druid/segment/indexing/DataSchemaTest.java @@ -72,7 +72,7 @@ public void testDefaultExclusions() throws Exception ); Assert.assertEquals( - ImmutableSet.of("time", "col1", "col2"), + ImmutableSet.of("time", "col1", "col2", "metric1", "metric2"), schema.getParser().getParseSpec().getDimensionsSpec().getDimensionExclusions() ); } @@ -101,7 +101,7 @@ public void testExplicitInclude() throws Exception ); Assert.assertEquals( - ImmutableSet.of("dimC", "col1"), + ImmutableSet.of("dimC", "col1", "metric1", "metric2"), schema.getParser().getParseSpec().getDimensionsSpec().getDimensionExclusions() ); } diff --git a/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java b/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java index ebdc7aca3e57..38c18da0ef43 100644 --- a/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java +++ b/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java @@ -20,6 +20,8 @@ import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.ObjectMapper; import com.metamx.common.Granularity; +import io.druid.client.cache.CacheConfig; +import io.druid.client.cache.MapCache; import io.druid.data.input.impl.DimensionsSpec; import io.druid.data.input.impl.JSONParseSpec; import io.druid.data.input.impl.StringInputRowParser; @@ -43,6 +45,22 @@ */ public class FireDepartmentTest { + + public static final CacheConfig NO_CACHE_CONFIG = new CacheConfig() + { + @Override + public boolean isPopulateCache() + { + return false; + } + + @Override + public boolean isUseCache() + { + return false; + } + }; + @Test public void testSerde() throws Exception { @@ -78,7 +96,16 @@ public void testSerde() throws Exception new RealtimeIOConfig( null, new RealtimePlumberSchool( - null, null, null, null, null, null, null + null, + null, + null, + null, + null, + null, + null, + MapCache.create(0), + NO_CACHE_CONFIG, + new DefaultObjectMapper() ), null ), diff --git a/server/src/test/java/io/druid/segment/realtime/firehose/EventReceiverFirehoseTest.java b/server/src/test/java/io/druid/segment/realtime/firehose/EventReceiverFirehoseTest.java new file mode 100644 index 000000000000..9df0170f70c7 --- /dev/null +++ b/server/src/test/java/io/druid/segment/realtime/firehose/EventReceiverFirehoseTest.java @@ -0,0 +1,220 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.segment.realtime.firehose; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +import com.metamx.common.ISE; +import io.druid.concurrent.Execs; +import io.druid.data.input.impl.DimensionsSpec; +import io.druid.data.input.impl.JSONParseSpec; +import io.druid.data.input.impl.MapInputRowParser; +import io.druid.data.input.impl.TimestampSpec; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.server.metrics.EventReceiverFirehoseMetric; +import io.druid.server.metrics.EventReceiverFirehoseRegister; +import org.apache.commons.io.IOUtils; +import org.easymock.EasyMock; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import javax.servlet.http.HttpServletRequest; +import java.io.IOException; +import java.io.InputStream; +import java.util.Map; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +public class EventReceiverFirehoseTest +{ + private static final int CAPACITY = 300; + private static final int NUM_EVENTS = 100; + private static final String SERVICE_NAME = "test_firehose"; + + private final String inputRow = "[{\n" + + " \"timestamp\":123,\n" + + " \"d1\":\"v1\"\n" + + "}]"; + + private EventReceiverFirehoseFactory eventReceiverFirehoseFactory; + private EventReceiverFirehoseFactory.EventReceiverFirehose firehose; + private EventReceiverFirehoseRegister register = new EventReceiverFirehoseRegister(); + private HttpServletRequest req; + + @Before + public void setUp() throws Exception + { + req = EasyMock.createMock(HttpServletRequest.class); + eventReceiverFirehoseFactory = new EventReceiverFirehoseFactory( + SERVICE_NAME, + CAPACITY, + null, + new DefaultObjectMapper(), + new DefaultObjectMapper(), + register + ); + firehose = (EventReceiverFirehoseFactory.EventReceiverFirehose) eventReceiverFirehoseFactory.connect( + new MapInputRowParser( + new JSONParseSpec( + new TimestampSpec( + "timestamp", + "auto", + null + ), new DimensionsSpec(ImmutableList.of("d1"), null, null) + ) + ) + ); + } + + @Test + public void testSingleThread() throws IOException + { + EasyMock.expect(req.getContentType()).andReturn("application/json").times(NUM_EVENTS); + EasyMock.replay(req); + + for (int i = 0; i < NUM_EVENTS; ++i) { + final InputStream inputStream = IOUtils.toInputStream(inputRow); + firehose.addAll(inputStream, req); + Assert.assertEquals(i + 1, firehose.getCurrentBufferSize()); + inputStream.close(); + } + + EasyMock.verify(req); + + final Iterable> metrics = register.getMetrics(); + Assert.assertEquals(1, Iterables.size(metrics)); + + final Map.Entry entry = Iterables.getLast(metrics); + Assert.assertEquals(SERVICE_NAME, entry.getKey()); + Assert.assertEquals(CAPACITY, entry.getValue().getCapacity()); + Assert.assertEquals(CAPACITY, firehose.getCapacity()); + Assert.assertEquals(NUM_EVENTS, entry.getValue().getCurrentBufferSize()); + Assert.assertEquals(NUM_EVENTS, firehose.getCurrentBufferSize()); + + for (int i = NUM_EVENTS - 1; i >= 0; --i) { + Assert.assertTrue(firehose.hasMore()); + Assert.assertNotNull(firehose.nextRow()); + Assert.assertEquals(i, firehose.getCurrentBufferSize()); + } + + Assert.assertEquals(CAPACITY, entry.getValue().getCapacity()); + Assert.assertEquals(CAPACITY, firehose.getCapacity()); + Assert.assertEquals(0, entry.getValue().getCurrentBufferSize()); + Assert.assertEquals(0, firehose.getCurrentBufferSize()); + + firehose.close(); + Assert.assertFalse(firehose.hasMore()); + Assert.assertEquals(0, Iterables.size(register.getMetrics())); + + } + + @Test + public void testMultipleThreads() throws InterruptedException, IOException, TimeoutException, ExecutionException + { + EasyMock.expect(req.getContentType()).andReturn("application/json").times(2 * NUM_EVENTS); + EasyMock.replay(req); + + final ExecutorService executorService = Execs.singleThreaded("single_thread"); + final Future future = executorService.submit( + new Callable() + { + @Override + public Boolean call() throws Exception + { + for (int i = 0; i < NUM_EVENTS; ++i) { + final InputStream inputStream = IOUtils.toInputStream(inputRow); + firehose.addAll(inputStream, req); + inputStream.close(); + } + return true; + } + } + ); + + for (int i = 0; i < NUM_EVENTS; ++i) { + final InputStream inputStream = IOUtils.toInputStream(inputRow); + firehose.addAll(inputStream, req); + inputStream.close(); + } + + future.get(10, TimeUnit.SECONDS); + + EasyMock.verify(req); + + final Iterable> metrics = register.getMetrics(); + Assert.assertEquals(1, Iterables.size(metrics)); + + final Map.Entry entry = Iterables.getLast(metrics); + + Assert.assertEquals(SERVICE_NAME, entry.getKey()); + Assert.assertEquals(CAPACITY, entry.getValue().getCapacity()); + Assert.assertEquals(CAPACITY, firehose.getCapacity()); + Assert.assertEquals(2 * NUM_EVENTS, entry.getValue().getCurrentBufferSize()); + Assert.assertEquals(2 * NUM_EVENTS, firehose.getCurrentBufferSize()); + + for (int i = 2 * NUM_EVENTS - 1; i >= 0; --i) { + Assert.assertTrue(firehose.hasMore()); + Assert.assertNotNull(firehose.nextRow()); + Assert.assertEquals(i, firehose.getCurrentBufferSize()); + } + + Assert.assertEquals(CAPACITY, entry.getValue().getCapacity()); + Assert.assertEquals(CAPACITY, firehose.getCapacity()); + Assert.assertEquals(0, entry.getValue().getCurrentBufferSize()); + Assert.assertEquals(0, firehose.getCurrentBufferSize()); + + firehose.close(); + Assert.assertFalse(firehose.hasMore()); + Assert.assertEquals(0, Iterables.size(register.getMetrics())); + + executorService.shutdownNow(); + } + + @Test(expected = ISE.class) + public void testDuplicateRegistering() throws IOException + { + EventReceiverFirehoseFactory eventReceiverFirehoseFactory2 = new EventReceiverFirehoseFactory( + SERVICE_NAME, + CAPACITY, + null, + new DefaultObjectMapper(), + new DefaultObjectMapper(), + register + ); + EventReceiverFirehoseFactory.EventReceiverFirehose firehose2 = + (EventReceiverFirehoseFactory.EventReceiverFirehose) eventReceiverFirehoseFactory2 + .connect( + new MapInputRowParser( + new JSONParseSpec( + new TimestampSpec( + "timestamp", + "auto", + null + ), new DimensionsSpec(ImmutableList.of("d1"), null, null) + ) + ) + ); + } +} diff --git a/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java b/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java index d298249cbf3a..f478ed9ad0ca 100644 --- a/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java +++ b/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java @@ -30,14 +30,12 @@ import com.metamx.emitter.service.ServiceEmitter; import io.druid.client.FilteredServerView; import io.druid.client.ServerView; -import io.druid.common.utils.JodaUtils; +import io.druid.client.cache.MapCache; import io.druid.data.input.Committer; import io.druid.data.input.InputRow; import io.druid.data.input.Row; import io.druid.data.input.impl.DimensionsSpec; -import io.druid.data.input.impl.InputRowParser; import io.druid.data.input.impl.JSONParseSpec; -import io.druid.data.input.impl.ParseSpec; import io.druid.data.input.impl.StringInputRowParser; import io.druid.data.input.impl.TimestampSpec; import io.druid.granularity.QueryGranularity; @@ -52,12 +50,12 @@ import io.druid.segment.indexing.granularity.UniformGranularitySpec; import io.druid.segment.loading.DataSegmentPusher; import io.druid.segment.realtime.FireDepartmentMetrics; +import io.druid.segment.realtime.FireDepartmentTest; import io.druid.segment.realtime.FireHydrant; import io.druid.segment.realtime.SegmentPublisher; import io.druid.server.coordination.DataSegmentAnnouncer; import io.druid.timeline.DataSegment; import org.apache.commons.io.FileUtils; -import org.apache.commons.lang.mutable.MutableBoolean; import org.easymock.EasyMock; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -198,7 +196,10 @@ public void setUp() throws Exception announcer, segmentPublisher, serverView, - MoreExecutors.sameThreadExecutor() + MoreExecutors.sameThreadExecutor(), + MapCache.create(0), + FireDepartmentTest.NO_CACHE_CONFIG, + new DefaultObjectMapper() ); metrics = new FireDepartmentMetrics(); diff --git a/server/src/test/java/io/druid/server/AsyncQueryForwardingServletTest.java b/server/src/test/java/io/druid/server/AsyncQueryForwardingServletTest.java index a9e3191efe55..23e41fd93a3a 100644 --- a/server/src/test/java/io/druid/server/AsyncQueryForwardingServletTest.java +++ b/server/src/test/java/io/druid/server/AsyncQueryForwardingServletTest.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; +import com.google.common.net.HostAndPort; import com.google.inject.Binder; import com.google.inject.Inject; import com.google.inject.Injector; @@ -49,16 +50,22 @@ import org.eclipse.jetty.server.handler.HandlerList; import org.eclipse.jetty.servlet.DefaultServlet; import org.eclipse.jetty.servlet.ServletContextHandler; +import org.eclipse.jetty.servlet.ServletHandler; import org.eclipse.jetty.servlet.ServletHolder; import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import javax.servlet.ServletException; +import javax.servlet.http.HttpServlet; import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; import java.io.IOException; import java.net.HttpURLConnection; import java.net.URI; import java.net.URL; +import java.util.Collection; +import java.util.concurrent.CountDownLatch; public class AsyncQueryForwardingServletTest extends BaseJettyTest { @@ -122,6 +129,40 @@ public void testProxyGzipCompression() throws Exception Assert.assertNotEquals("gzip", postNoGzip.getContentEncoding()); } + @Test(timeout = 60_000) + public void testDeleteBroadcast() throws Exception + { + CountDownLatch latch = new CountDownLatch(2); + makeTestDeleteServer(port + 1, latch).start(); + makeTestDeleteServer(port + 2, latch).start(); + + final URL url = new URL("http://localhost:" + port + "/druid/v2/abc123"); + final HttpURLConnection post = (HttpURLConnection) url.openConnection(); + post.setRequestMethod("DELETE"); + int code = post.getResponseCode(); + Assert.assertEquals(200, code); + + latch.await(); + } + + private static Server makeTestDeleteServer(int port, final CountDownLatch latch) + { + Server server = new Server(port); + ServletHandler handler = new ServletHandler(); + handler.addServletWithMapping(new ServletHolder(new HttpServlet() + { + @Override + protected void doDelete(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException + { + latch.countDown(); + resp.setStatus(200); + } + }), "/default/*"); + + server.setHandler(handler); + return server; + } + public static class ProxyJettyServerInit implements JettyServerInitializer { @@ -152,6 +193,16 @@ public String getDefaultHost() { return "localhost:" + node.getPort(); } + + @Override + public Collection getAllHosts() + { + return ImmutableList.of( + "localhost:" + node.getPort(), + "localhost:" + (node.getPort() + 1), + "localhost:" + (node.getPort() + 2) + ); + } }; ServletHolder holder = new ServletHolder( @@ -173,15 +224,19 @@ public void log(RequestLogLine requestLogLine) throws IOException ) { @Override - protected URI rewriteURI(HttpServletRequest request) + protected URI rewriteURI(HttpServletRequest request, String host) { - URI uri = super.rewriteURI(request); - return URI.create(uri.toString().replace("/proxy", "")); + String uri = super.rewriteURI(request, host).toString(); + if (uri.contains("/druid/v2")) { + return URI.create(uri.replace("/druid/v2", "/default")); + } + return URI.create(uri.replace("/proxy", "")); } }); //NOTE: explicit maxThreads to workaround https://tickets.puppetlabs.com/browse/TK-152 holder.setInitParameter("maxThreads", "256"); root.addServlet(holder, "/proxy/*"); + root.addServlet(holder, "/druid/v2/*"); JettyServerInitUtils.addExtensionFilters(root, injector); root.addFilter(JettyServerInitUtils.defaultAsyncGzipFilterHolder(), "/*", null); root.addFilter(GuiceFilter.class, "/slow/*", null); @@ -193,4 +248,32 @@ protected URI rewriteURI(HttpServletRequest request) server.setHandler(handlerList); } } + + @Test + public void testRewriteURI() throws Exception + { + + // test params + Assert.assertEquals( + new URI("http://localhost:1234/some/path?param=1"), + AsyncQueryForwardingServlet.makeURI("localhost:1234", "/some/path", "param=1") + ); + + // HttpServletRequest.getQueryString returns encoded form + // use ascii representation in case URI is using non-ascii characters + Assert.assertEquals( + "http://[2a00:1450:4007:805::1007]:1234/some/path?param=1¶m2=%E2%82%AC", + AsyncQueryForwardingServlet.makeURI( + HostAndPort.fromParts("2a00:1450:4007:805::1007", 1234).toString(), + "/some/path", + "param=1¶m2=%E2%82%AC" + ).toASCIIString() + ); + + // test null query + Assert.assertEquals( + new URI("http://localhost/"), + AsyncQueryForwardingServlet.makeURI("localhost", "/", null) + ); + } } diff --git a/server/src/test/java/io/druid/server/initialization/BaseJettyTest.java b/server/src/test/java/io/druid/server/initialization/BaseJettyTest.java index 9a5d6f4603bf..fc4867a25af8 100644 --- a/server/src/test/java/io/druid/server/initialization/BaseJettyTest.java +++ b/server/src/test/java/io/druid/server/initialization/BaseJettyTest.java @@ -64,6 +64,7 @@ import javax.servlet.ServletResponse; import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; +import javax.ws.rs.DELETE; import javax.ws.rs.GET; import javax.ws.rs.POST; import javax.ws.rs.Path; @@ -236,6 +237,14 @@ public Response hello() @Path("/default") public static class DefaultResource { + @DELETE + @Path("{resource}") + @Produces(MediaType.APPLICATION_JSON) + public Response delete() + { + return Response.ok("hello").build(); + } + @GET @Produces(MediaType.APPLICATION_JSON) public Response get() diff --git a/services/src/main/java/io/druid/cli/CliPeon.java b/services/src/main/java/io/druid/cli/CliPeon.java index 7c4053907361..5a2b3bbf9ff9 100644 --- a/services/src/main/java/io/druid/cli/CliPeon.java +++ b/services/src/main/java/io/druid/cli/CliPeon.java @@ -30,7 +30,9 @@ import io.airlift.airline.Arguments; import io.airlift.airline.Command; import io.airlift.airline.Option; +import io.druid.client.cache.CacheConfig; import io.druid.guice.Binders; +import io.druid.guice.CacheModule; import io.druid.guice.IndexingServiceFirehoseModule; import io.druid.guice.Jerseys; import io.druid.guice.JsonConfigProvider; @@ -70,6 +72,7 @@ import io.druid.segment.realtime.firehose.NoopChatHandlerProvider; import io.druid.segment.realtime.firehose.ServiceAnnouncingChatHandlerProvider; import io.druid.server.QueryResource; +import io.druid.server.initialization.jetty.ChatHandlerServerModule; import io.druid.server.initialization.jetty.JettyServerInitializer; import org.eclipse.jetty.server.Server; @@ -156,6 +159,9 @@ public void configure(Binder binder) binder.bind(QuerySegmentWalker.class).to(ThreadPoolTaskRunner.class); binder.bind(ThreadPoolTaskRunner.class).in(ManageLifecycle.class); + JsonConfigProvider.bind(binder, "druid.realtime.cache", CacheConfig.class); + binder.install(new CacheModule()); + // Override the default SegmentLoaderConfig because we don't actually care about the // configuration based locations. This will override them anyway. This is also stopping // configuration of other parameters, but I don't think that's actually a problem. @@ -198,7 +204,8 @@ private void configureTaskActionClient(Binder binder) } }, - new IndexingServiceFirehoseModule() + new IndexingServiceFirehoseModule(), + new ChatHandlerServerModule() ); } diff --git a/services/src/main/java/io/druid/cli/CliRealtime.java b/services/src/main/java/io/druid/cli/CliRealtime.java index d95cf079a69a..0bf2fd844b67 100644 --- a/services/src/main/java/io/druid/cli/CliRealtime.java +++ b/services/src/main/java/io/druid/cli/CliRealtime.java @@ -24,6 +24,7 @@ import com.metamx.common.logger.Logger; import io.airlift.airline.Command; import io.druid.guice.RealtimeModule; +import io.druid.server.initialization.jetty.ChatHandlerServerModule; import java.util.List; @@ -54,7 +55,8 @@ public void configure(Binder binder) binder.bindConstant().annotatedWith(Names.named("serviceName")).to("druid/realtime"); binder.bindConstant().annotatedWith(Names.named("servicePort")).to(8084); } - } + }, + new ChatHandlerServerModule() ); } } diff --git a/services/src/main/java/io/druid/cli/Log4JShutdownPropertyChecker.java b/services/src/main/java/io/druid/cli/Log4JShutdownPropertyChecker.java new file mode 100644 index 000000000000..37f959a9341b --- /dev/null +++ b/services/src/main/java/io/druid/cli/Log4JShutdownPropertyChecker.java @@ -0,0 +1,36 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.cli; + +import java.util.Properties; + +public class Log4JShutdownPropertyChecker implements PropertyChecker +{ + @Override + public void checkProperties(Properties properties) + { + if (!properties.contains("log4j.shutdownCallbackRegistry")) { + properties.setProperty("log4j.shutdownCallbackRegistry", "io.druid.common.config.Log4jShutdown"); + } + if (!properties.contains("log4j.shutdownHookEnabled")) { + properties.setProperty("log4j.shutdownHookEnabled", "true"); + } + } +} diff --git a/services/src/main/java/io/druid/cli/Main.java b/services/src/main/java/io/druid/cli/Main.java index 113bdf19c316..9f7fddd936fd 100644 --- a/services/src/main/java/io/druid/cli/Main.java +++ b/services/src/main/java/io/druid/cli/Main.java @@ -28,11 +28,19 @@ import io.druid.initialization.Initialization; import java.util.Collection; +import java.util.ServiceLoader; /** */ public class Main { + static { + ServiceLoader serviceLoader = ServiceLoader.load(PropertyChecker.class); + for (PropertyChecker propertyChecker : serviceLoader) { + propertyChecker.checkProperties(System.getProperties()); + } + } + @SuppressWarnings("unchecked") public static void main(String[] args) { diff --git a/services/src/main/java/io/druid/cli/PropertyChecker.java b/services/src/main/java/io/druid/cli/PropertyChecker.java new file mode 100644 index 000000000000..a32bc14d6833 --- /dev/null +++ b/services/src/main/java/io/druid/cli/PropertyChecker.java @@ -0,0 +1,39 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.cli; + +import java.util.Properties; + +/** + * The PropertyChecker classes are loaded by ServiceLoader at the very start of the program and as such MUST be on the + * initial classpath and cannot be loaded via extensions at runtime. (Or more precisely, they are ignored if present + * in an extension at runtime, but not on the initial classpath) + * + * The PropertyChecker should ONLY try and set very specific properties and any class loading should be done in an + * isolated class loader to not pollute the general class loader + */ +public interface PropertyChecker +{ + /** + * Check the given properties to make sure any unset values are properly configured. + * @param properties The properties to check, usually System.getProperties() + */ + void checkProperties(Properties properties); +} diff --git a/services/src/main/java/io/druid/guice/RealtimeModule.java b/services/src/main/java/io/druid/guice/RealtimeModule.java index 784aa7958be2..c0753fe01fa2 100644 --- a/services/src/main/java/io/druid/guice/RealtimeModule.java +++ b/services/src/main/java/io/druid/guice/RealtimeModule.java @@ -23,6 +23,7 @@ import com.google.inject.TypeLiteral; import com.google.inject.multibindings.MapBinder; import io.druid.cli.QueryJettyServerInitializer; +import io.druid.client.cache.CacheConfig; import io.druid.metadata.MetadataSegmentPublisher; import io.druid.query.QuerySegmentWalker; import io.druid.segment.realtime.FireDepartment; @@ -35,7 +36,6 @@ import io.druid.segment.realtime.firehose.ServiceAnnouncingChatHandlerProvider; import io.druid.server.QueryResource; import io.druid.server.initialization.jetty.JettyServerInitializer; - import org.eclipse.jetty.server.Server; import java.util.List; @@ -44,6 +44,7 @@ */ public class RealtimeModule implements Module { + @Override public void configure(Binder binder) { @@ -80,6 +81,9 @@ public void configure(Binder binder) .toProvider(FireDepartmentsProvider.class) .in(LazySingleton.class); + JsonConfigProvider.bind(binder, "druid.realtime.cache", CacheConfig.class); + binder.install(new CacheModule()); + binder.bind(QuerySegmentWalker.class).to(RealtimeManager.class).in(ManageLifecycle.class); binder.bind(NodeTypeConfig.class).toInstance(new NodeTypeConfig("realtime")); binder.bind(JettyServerInitializer.class).to(QueryJettyServerInitializer.class).in(LazySingleton.class); diff --git a/services/src/main/resources/META-INF/services/io.druid.cli.PropertyChecker b/services/src/main/resources/META-INF/services/io.druid.cli.PropertyChecker new file mode 100644 index 000000000000..6c4bd09dfec9 --- /dev/null +++ b/services/src/main/resources/META-INF/services/io.druid.cli.PropertyChecker @@ -0,0 +1,20 @@ +# +# Licensed to Metamarkets Group Inc. (Metamarkets) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. Metamarkets licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +# + +io.druid.cli.Log4JShutdownPropertyChecker