diff --git a/.gitignore b/.gitignore index f906e242679..89eabfcbdd0 100644 --- a/.gitignore +++ b/.gitignore @@ -25,5 +25,5 @@ README .pmdruleset.xml .java-version integration-tests/gen-scripts/ -/bin/ +bin/ *.hprof diff --git a/.travis.yml b/.travis.yml index 9ca6fee1ad7..14d16e5e363 100644 --- a/.travis.yml +++ b/.travis.yml @@ -46,7 +46,7 @@ addons: # Add various options to make 'mvn install' fast and skip javascript compile (-Ddruid.console.skip=true) since it is not # needed. Depending on network speeds, "mvn -q install" may take longer than the default 10 minute timeout to print any # output. To compensate, use travis_wait to extend the timeout. -install: ./check_test_suite.py && travis_terminate 0 || echo 'Running Maven install...' && MAVEN_OPTS='-Xmx3000m' travis_wait 15 ${MVN} clean install -q -ff -pl '!distribution,!:it-tools,!:it-image' ${MAVEN_SKIP} ${MAVEN_SKIP_TESTS} -T1C && ${MVN} install -q -ff -pl 'distribution' ${MAVEN_SKIP} ${MAVEN_SKIP_TESTS} +install: ./check_test_suite.py && travis_terminate 0 || echo 'Running Maven install...' && MAVEN_OPTS='-Xmx3000m' travis_wait 15 ${MVN} clean install -q -ff -pl '!distribution,!:druid-it-tools,!:druid-it-image,!:druid-it-cases' ${MAVEN_SKIP} ${MAVEN_SKIP_TESTS} -T1C && ${MVN} install -q -ff -pl 'distribution' ${MAVEN_SKIP} ${MAVEN_SKIP_TESTS} # There are 3 stages of tests # 1. Tests - phase 1 @@ -72,7 +72,7 @@ jobs: - name: "animal sniffer checks" stage: Tests - phase 1 script: ${MVN} animal-sniffer:check --fail-at-end - + - name: "checkstyle" script: ${MVN} checkstyle:checkstyle --fail-at-end @@ -347,7 +347,7 @@ jobs: <<: *test_processing_module name: "(openjdk8) other modules test" env: - - MAVEN_PROJECTS='!processing,!indexing-hadoop,!indexing-service,!extensions-core/kafka-indexing-service,!extensions-core/kinesis-indexing-service,!server,!web-console,!integration-tests,!:it-image,!:it-tools' + - MAVEN_PROJECTS='!processing,!indexing-hadoop,!indexing-service,!extensions-core/kafka-indexing-service,!extensions-core/kinesis-indexing-service,!server,!web-console,!integration-tests,!:druid-it-tools,!:druid-it-image,!:druid-it-cases' - <<: *test_other_modules name: "(openjdk11) other modules test" @@ -457,9 +457,9 @@ jobs: docker exec -it druid-$v sh -c 'dmesg | tail -3' ; done - - <<: *integration_batch_index - name: "(Compile=openjdk8, Run=openjdk8) batch index integration test with Indexer" - env: TESTNG_GROUPS='-Dgroups=batch-index' JVM_RUNTIME='-Djvm.runtime=8' USE_INDEXER='indexer' + #- <<: *integration_batch_index + # name: "(Compile=openjdk8, Run=openjdk8) batch index integration test with Indexer" + # env: TESTNG_GROUPS='-Dgroups=batch-index' JVM_RUNTIME='-Djvm.runtime=8' USE_INDEXER='indexer' - &integration_input_format name: "(Compile=openjdk8, Run=openjdk8) input format integration test" @@ -666,16 +666,33 @@ jobs: name: "(Compile=openjdk8, Run=openjdk8) other integration tests with Indexer" env: TESTNG_GROUPS='-DexcludedGroups=batch-index,input-format,input-source,perfect-rollup-parallel-batch-index,kafka-index,query,query-retry,query-error,realtime-index,security,ldap-security,s3-deep-storage,gcs-deep-storage,azure-deep-storage,hdfs-deep-storage,s3-ingestion,kinesis-index,kinesis-data-format,kafka-transactional-index,kafka-index-slow,kafka-transactional-index-slow,kafka-data-format,hadoop-s3-to-s3-deep-storage,hadoop-s3-to-hdfs-deep-storage,hadoop-azure-to-azure-deep-storage,hadoop-azure-to-hdfs-deep-storage,hadoop-gcs-to-gcs-deep-storage,hadoop-gcs-to-hdfs-deep-storage,aliyun-oss-deep-storage,append-ingestion,compaction,high-availability,upgrade,shuffle-deep-store,custom-coordinator-duties' JVM_RUNTIME='-Djvm.runtime=8' USE_INDEXER='indexer' - - <<: *integration_tests - name: "(Compile=openjdk8, Run=openjdk8) leadership and high availability integration tests" - jdk: openjdk8 - env: TESTNG_GROUPS='-Dgroups=high-availability' JVM_RUNTIME='-Djvm.runtime=8' USE_INDEXER='middleManager' OVERRIDE_CONFIG_PATH='./environment-configs/test-groups/prepopulated-data' + #- <<: *integration_tests + # name: "(Compile=openjdk8, Run=openjdk8) leadership and high availability integration tests" + # jdk: openjdk8 + # env: TESTNG_GROUPS='-Dgroups=high-availability' JVM_RUNTIME='-Djvm.runtime=8' USE_INDEXER='middleManager' OVERRIDE_CONFIG_PATH='./environment-configs/test-groups/prepopulated-data' - <<: *integration_query name: "(Compile=openjdk8, Run=openjdk8) query integration test (mariaDB)" jdk: openjdk8 env: TESTNG_GROUPS='-Dgroups=query' USE_INDEXER='middleManager' MYSQL_DRIVER_CLASSNAME='org.mariadb.jdbc.Driver' OVERRIDE_CONFIG_PATH='./environment-configs/test-groups/prepopulated-data' + # Revised ITs. + - &integration_tests_ex + name: "(Compile=openjdk8, Run=openjdk8) leadership and high availability integration tests (new)" + stage: Tests - phase 2 + jdk: openjdk8 + services: *integration_test_services + env: JVM_RUNTIME='-Djvm.runtime=8' USE_INDEXER='middleManager' + # Uses the install defined above. Then, builds the test tools and docker image, + # and runs one IT. If tests fail, echos log lines of any of + # the Druid services that did not exit normally. + script: ./it.sh travis HighAvailability + + - <<: *integration_tests_ex + name: "(Compile=openjdk8, Run=openjdk8) batch index integration test with Indexer (new)" + env: JVM_RUNTIME='-Djvm.runtime=8' USE_INDEXER='indexer' + script: ./it.sh travis BatchIndex + # END - Integration tests for Compile with Java 8 and Run with Java 8 # START - Integration tests for Compile with Java 8 and Run with Java 11 @@ -756,21 +773,6 @@ jobs: # END - Integration tests for Compile with Java 8 and Run with Java 11 - # BEGIN - Revised integration tests - - # Experimental build of the revised integration test Docker image. - # Actual tests will come later. - - name: "experimental docker tests" - stage: Tests - phase 2 - # Uses the install defined above. Then, builds the test tools and docker image, - # and run the various IT tests. If tests fail, echos log lines of any of - # the Druid services that did not exit normally. - # Run though install to ensure the test tools are installed, and the docker - # image is built. The tests only need verify. - script: ${MVN} install -P dist,test-image -rf :distribution ${MAVEN_SKIP} -DskipUTs=true - - # END - Revised integration tests - - &integration_batch_index_k8s name: "(Compile=openjdk8, Run=openjdk8, Cluster Build On K8s) ITNestedQueryPushDownTest integration test" stage: Tests - phase 2 diff --git a/core/src/main/java/org/apache/druid/guice/PolyBind.java b/core/src/main/java/org/apache/druid/guice/PolyBind.java index e7e53b6c984..19931a6b63e 100644 --- a/core/src/main/java/org/apache/druid/guice/PolyBind.java +++ b/core/src/main/java/org/apache/druid/guice/PolyBind.java @@ -40,12 +40,13 @@ import java.util.Map; import java.util.Properties; /** - * Provides the ability to create "polymorphic" bindings. Where the polymorphism is actually just making a decision - * based on a value in a Properties. - * - * The workflow is that you first create a choice by calling createChoice(). Then you create options using the binder - * returned by the optionBinder() method. Multiple different modules can call optionBinder and all options will be - * reflected at injection time as long as equivalent interface Key objects are passed into the various methods. + * Provides the ability to create "polymorphic" bindings where the polymorphism is actually just making a decision + * based on a value in Properties. + *

+ * The workflow is that you first create a choice by calling {@code createChoice()}. Then you create options using + * the binder returned by the {@code optionBinder()} method. Multiple different modules can call + * {@code optionBinder()} and all options will be reflected at injection time as long as equivalent interface + * {@code Key} objects are passed into the various methods. */ @PublicApi public class PolyBind @@ -110,10 +111,10 @@ public class PolyBind } /** - * Binds an option for a specific choice. The choice must already be registered on the injector for this to work. + * Binds an option for a specific choice. The choice must already be registered on the injector for this to work. * * @param binder the binder for the injector that is being configured - * @param interfaceKey the interface that will have an option added to it. This must equal the + * @param interfaceKey the interface that will have an option added to it. This must equal the * Key provided to createChoice * @param interface type * @return A MapBinder that can be used to create the actual option bindings. @@ -195,7 +196,7 @@ public class PolyBind if (provider == null) { throw new ProvisionException( - StringUtils.format("Unknown provider[%s] of %s, known options[%s]", implName, key, implsMap.keySet()) + StringUtils.format("Unknown provider [%s] of %s, known options [%s]", implName, key, implsMap.keySet()) ); } diff --git a/core/src/main/java/org/apache/druid/metadata/MetadataStorageConnectorConfig.java b/core/src/main/java/org/apache/druid/metadata/MetadataStorageConnectorConfig.java index 512cd57d892..2e6767f0b77 100644 --- a/core/src/main/java/org/apache/druid/metadata/MetadataStorageConnectorConfig.java +++ b/core/src/main/java/org/apache/druid/metadata/MetadataStorageConnectorConfig.java @@ -29,6 +29,8 @@ import java.util.Properties; */ public class MetadataStorageConnectorConfig { + public static final String PROPERTY_BASE = "druid.metadata.storage.connector"; + @JsonProperty private boolean createTables = true; diff --git a/core/src/main/java/org/apache/druid/metadata/MetadataStorageTablesConfig.java b/core/src/main/java/org/apache/druid/metadata/MetadataStorageTablesConfig.java index 766efabb526..267f4fe6b69 100644 --- a/core/src/main/java/org/apache/druid/metadata/MetadataStorageTablesConfig.java +++ b/core/src/main/java/org/apache/druid/metadata/MetadataStorageTablesConfig.java @@ -30,6 +30,8 @@ import java.util.Map; */ public class MetadataStorageTablesConfig { + public static final String PROPERTY_BASE = "druid.metadata.storage.tables"; + public static MetadataStorageTablesConfig fromBase(String base) { return new MetadataStorageTablesConfig(base, null, null, null, null, null, null, null, null, null, null); diff --git a/core/src/test/java/org/apache/druid/data/input/impl/prefetch/PrefetchableTextFilesFirehoseFactoryTest.java b/core/src/test/java/org/apache/druid/data/input/impl/prefetch/PrefetchableTextFilesFirehoseFactoryTest.java index dd9c384e91a..38fb843a34c 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/prefetch/PrefetchableTextFilesFirehoseFactoryTest.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/prefetch/PrefetchableTextFilesFirehoseFactoryTest.java @@ -41,6 +41,7 @@ import org.hamcrest.CoreMatchers; import org.junit.Assert; import org.junit.BeforeClass; import org.junit.ClassRule; +import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; @@ -249,6 +250,7 @@ public class PrefetchableTextFilesFirehoseFactoryTest } @Test + @Ignore("See issue #12638") public void testWithLargeCacheAndSmallFetch() throws IOException { final TestPrefetchableTextFilesFirehoseFactory factory = @@ -336,6 +338,7 @@ public class PrefetchableTextFilesFirehoseFactoryTest } @Test + @Ignore("See issue #12638") public void testReconnectWithCacheAndPrefetch() throws IOException { final TestPrefetchableTextFilesFirehoseFactory factory = diff --git a/core/src/test/java/org/apache/druid/guice/PolyBindTest.java b/core/src/test/java/org/apache/druid/guice/PolyBindTest.java index d6a50d51396..372d428458d 100644 --- a/core/src/test/java/org/apache/druid/guice/PolyBindTest.java +++ b/core/src/test/java/org/apache/druid/guice/PolyBindTest.java @@ -112,7 +112,7 @@ public class PolyBindTest } catch (Exception e) { Assert.assertTrue(e instanceof ProvisionException); - Assert.assertTrue(e.getMessage().contains("Unknown provider[c] of Key[type=org.apache.druid.guice.PolyBindTest$Gogo")); + Assert.assertTrue(e.getMessage().contains("Unknown provider [c] of Key[type=org.apache.druid.guice.PolyBindTest$Gogo")); } try { Assert.assertEquals("B", injector.getInstance(Key.get(Gogo.class, Names.named("reverse"))).go()); @@ -120,9 +120,9 @@ public class PolyBindTest } catch (Exception e) { Assert.assertTrue(e instanceof ProvisionException); - Assert.assertTrue(e.getMessage().contains("Unknown provider[c] of Key[type=org.apache.druid.guice.PolyBindTest$Gogo")); + Assert.assertTrue(e.getMessage().contains("Unknown provider [c] of Key[type=org.apache.druid.guice.PolyBindTest$Gogo")); } - + // test default property value Assert.assertEquals("B", injector.getInstance(GogoSally.class).go()); props.setProperty("sally", "a"); @@ -136,7 +136,7 @@ public class PolyBindTest } catch (Exception e) { Assert.assertTrue(e instanceof ProvisionException); - Assert.assertTrue(e.getMessage().contains("Unknown provider[c] of Key[type=org.apache.druid.guice.PolyBindTest$GogoSally")); + Assert.assertTrue(e.getMessage().contains("Unknown provider [c] of Key[type=org.apache.druid.guice.PolyBindTest$GogoSally")); } } diff --git a/extensions-core/mysql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/mysql/MySQLConnector.java b/extensions-core/mysql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/mysql/MySQLConnector.java index bbe238ffb0f..4098e30a544 100644 --- a/extensions-core/mysql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/mysql/MySQLConnector.java +++ b/extensions-core/mysql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/mysql/MySQLConnector.java @@ -61,7 +61,7 @@ public class MySQLConnector extends SQLMetadataConnector ) { super(config, dbTables); - log.info("Loading \"MySQL\" metadata connector driver %s", driverConfig.getDriverClassName()); + log.info("Loading MySQL metadata connector driver %s", driverConfig.getDriverClassName()); tryLoadDriverClass(driverConfig.getDriverClassName(), true); if (driverConfig.getDriverClassName().contains("mysql")) { diff --git a/extensions-core/mysql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/mysql/MySQLConnectorDriverConfig.java b/extensions-core/mysql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/mysql/MySQLConnectorDriverConfig.java index 0f6fa03245d..4aad21651c7 100644 --- a/extensions-core/mysql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/mysql/MySQLConnectorDriverConfig.java +++ b/extensions-core/mysql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/mysql/MySQLConnectorDriverConfig.java @@ -25,8 +25,11 @@ import java.util.Objects; public class MySQLConnectorDriverConfig { + public static final String MYSQL_DRIVER = "com.mysql.jdbc.Driver"; + public static final String MARIA_DB_DRIVER = "org.mariadb.jdbc.Driver"; + @JsonProperty - private String driverClassName = "com.mysql.jdbc.Driver"; + private String driverClassName = MYSQL_DRIVER; @JsonProperty public String getDriverClassName() diff --git a/extensions-core/testing-tools/.gitignore b/extensions-core/testing-tools/.gitignore new file mode 100644 index 00000000000..ae3c1726048 --- /dev/null +++ b/extensions-core/testing-tools/.gitignore @@ -0,0 +1 @@ +/bin/ diff --git a/integration-tests-ex/README.md b/integration-tests-ex/README.md index 9619897baeb..9c29ec101fe 100644 --- a/integration-tests-ex/README.md +++ b/integration-tests-ex/README.md @@ -19,17 +19,116 @@ # Revised Integration Tests -This directory builds a Docker image for Druid. Later revisions -use the image to run revised integration tests. +This directory builds a Docker image for Druid, then uses that image, along +with test configuration to run tests. This version greatly evolves the +integration tests from the earlier form. See the [History](docs/history.md) +section for details. -The `it-base` project is built as part of the normal build, -though it is used only for the Docker image. +## Shortcuts -To build the image: +List of the most common commands once you're familiar with the framework. +If you are new to the framework, see [Quickstart](docs/quickstart.md) for +an explanation. + +### Build Druid + +To make the text a bit simpler, define a variable for the standard settings: ```bash -mvn $USUAL_CAVEATS -P test-image +export MAVEN_IGNORE=-P skip-static-checks,skip-tests -Dmaven.javadoc.skip=true + +```bash +mvn clean package -P dist $MAVEN_IGNORE -T1.0C ``` -Where `$USUAL_CAVEATS` are your favorite options to turn -off static checks, UTs, etc. \ No newline at end of file +### Build the Test Image + +```bash +cd $DRUID_DEV/integration-tests-ex/image +mvn install -P test-image $MAVEN_IGNORE +``` + +### Run an IT from the Command Line + +```bash +mvn verify -P IT- -pl :druid-it-cases $MAVEN_IGNORE +``` + +Where `` is one of the test categories. + +Or + +```bash +cd $DRUID_DEV/integration-tests-ex/cases +mvn verify -P skip-static-checks,docker-tests,IT- \ + -Dmaven.javadoc.skip=true -DskipUTs=true \ + -pl :druid-it-cases +``` + +### Run an IT from the IDE + +Start the cluster: + +```bash +cd $DRUID_DEV/integration-tests-ex/cases +./cluster.sh up +``` + +Where `` is one of the test categories. Then launch the +test as a JUnit test. + +## Contents + +* [Goals](#Goals) +* [Quickstart](docs/quickstart.md) +* [Create a new test](docs/guide.md) +* [Maven configuration](docs/maven.md) +* [Travis integration](docs/travis.md) +* [Docker image](docs/docker.md) +* [Druid configuration](docs/druid-config.md) +* [Docker Compose configuration](docs/compose.md) +* [Test configuration](docs/test-config.md) +* [Test structure](docs/tests.md) +* [Test runtime semantics](docs/runtime.md) +* [Scripts](docs/scripts.md) +* [Dependencies](docs/dependencies.md) +* [Debugging](docs/debugging.md) + +Background information + +* [Next steps](docs/next-steps.md) +* [Test conversion](docs/conversion.md) - How to convert existing tests. +* [History](docs/history.md) - Comparison with prior integration tests. + +## Goals + +The goal of the present version is to simplify development. + +* Speed up the Druid test image build by avoiding download of + dependencies. (Instead, any such dependencies are managed by + Maven and reside in the local build cache.) +* Use official images for dependencies to avoid the need to + download, install, and manage those dependencies. +* Make it is easy to manually build the image, launch + a cluster, and run a test against the cluster. +* Convert tests to JUnit so that they will easily run in your + favorite IDE, just like other Druid tests. +* Use the actual Druid build from `distribution` so we know + what is tested. +* Leverage, don't fight, Maven. +* Run the integration tests easily on a typical development machine. + +By meeting these goals, you can quickly: + +* Build the Druid distribution. +* Build the Druid image. (< 1 minute) +* Launch the cluster for the particular test. (a few seconds) +* Run the test any number of times in your debugger. +* Clean up the test artifacts. + +The result is that the fastest path to develop a Druid patch or +feature is: + +* Create a normal unit test and run it to verify your code. +* Create an integration test that double-checks the code in + a live cluster. diff --git a/integration-tests-ex/cases/.gitignore b/integration-tests-ex/cases/.gitignore new file mode 100644 index 00000000000..ae3c1726048 --- /dev/null +++ b/integration-tests-ex/cases/.gitignore @@ -0,0 +1 @@ +/bin/ diff --git a/integration-tests-ex/assets/log4j2.xml b/integration-tests-ex/cases/assets/log4j2.xml similarity index 100% rename from integration-tests-ex/assets/log4j2.xml rename to integration-tests-ex/cases/assets/log4j2.xml diff --git a/integration-tests-ex/cases/cluster.sh b/integration-tests-ex/cases/cluster.sh new file mode 100755 index 00000000000..0b19b478fcb --- /dev/null +++ b/integration-tests-ex/cases/cluster.sh @@ -0,0 +1,201 @@ +#! /bin/bash + +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +#-------------------------------------------------------------------- + +# Starts the test-specific test cluster using Docker compose using +# versions and other settings gathered when the test image was built. +# Maps category names to cluster names. The mapping here must match +# that in the test category classes when @Cluster is used. + +# Enable for debugging +#set -x + +export MODULE_DIR=$(cd $(dirname $0) && pwd) + +function usage { + cat <&2 + exit 1 +fi + +CMD=$1 +shift + +function category { + if [ $# -eq 0 ]; then + usage 1>&2 + exit 1 + fi + export CATEGORY=$1 + + # All commands need env vars + ENV_FILE=$MODULE_DIR/../image/target/env.sh + if [ ! -f $ENV_FILE ]; then + echo "Please build the Docker test image before testing" 1>&2 + exit 1 + fi + + source $ENV_FILE + # The untranslated category is used for the local name of the + # shared folder. + + # DRUID_INTEGRATION_TEST_GROUP is used in + # docker-compose files and here. Despite the name, it is the + # name of the cluster configuration we want to run, not the + # test category. Multiple categories an map to the same cluster + # definition. + + # Map from category name to shared cluster definition name. + # Add an entry here if you create a new category that shares + # a definition. + case $CATEGORY in + "InputFormat") + export DRUID_INTEGRATION_TEST_GROUP=BatchIndex + ;; + *) + export DRUID_INTEGRATION_TEST_GROUP=$CATEGORY + ;; + esac + + export CLUSTER_DIR=$MODULE_DIR/cluster/$DRUID_INTEGRATION_TEST_GROUP + if [ ! -d $CLUSTER_DIR ]; then + echo "Cluster directory $CLUSTER_DIR does not exist." 1>&2 + echo "$USAGE" 1>&2 + exit 1 + fi + + export TARGET_DIR=$MODULE_DIR/target + export SHARED_DIR=$TARGET_DIR/$CATEGORY + export ENV_FILE="$TARGET_DIR/${CATEGORY}.env" +} + +function build_override { + + mkdir -p target + rm -f "$ENV_FILE" + touch "$ENV_FILE" + + # User-local settings? + LOCAL_ENV="$HOME/druid-it/${CATEGORY}.env" + if [ -f "$LOCAL_ENV" ]; then + cat "$LOCAL_ENV" >> "$ENV_FILE" + fi + + # Provided override file + if [ -n "$OVERRIDE_ENV" ]; then + if [ ! -f "$OVERRIDE_ENV" ]; then + echo "Environment override file (OVERRIDE_ENV) not found: $OVERRIDE_ENV" 1>&2 + exit 1 + fi + cat "$OVERRIDE_ENV" >> "$ENV_FILE" + fi + + # Add all environment variables of the form druid_* + env | grep "^druid_" >> "$ENV_FILE" + + # Reuse the OVERRIDE_ENV variable to pass the full list to Docker compose + export OVERRIDE_ENV="$ENV_FILE" +} + +# Dump lots of information to debug Docker failures when run inside +# of a build environment where we can't inspect Docker directly. +function show_status { + echo "====================================" + ls -l target/shared + echo "docker ps -a" + docker ps -a + # Was: --filter status=exited + for id in $(docker ps -a --format "{{.ID}}"); do + echo "====================================" + echo "Logs for Container ID $id" + docker logs $id | tail -n 20 + done + echo "====================================" +} + +function build_shared_dir { + mkdir -p $SHARED_DIR + # Must start with an empty DB to keep MySQL happy + rm -rf $SHARED_DIR/db + mkdir -p $SHARED_DIR/logs + mkdir -p $SHARED_DIR/tasklogs + mkdir -p $SHARED_DIR/db + mkdir -p $SHARED_DIR/kafka + mkdir -p $SHARED_DIR/resources + cp $MODULE_DIR/assets/log4j2.xml $SHARED_DIR/resources + # Permissions in some build setups are screwed up. See above. The user + # which runs Docker does not have permission to write into the /shared + # directory. Force ownership to allow writing. + chmod -R a+rwx $SHARED_DIR +} + +# Print environment for debugging +#env + +case $CMD in + "-h" ) + usage + ;; + "help" ) + usage + docker-compose help + ;; + "up" ) + category $* + echo "Starting cluster $DRUID_INTEGRATION_TEST_GROUP" + build_override + build_shared_dir + cd $CLUSTER_DIR + docker-compose up -d + # Enable the following for debugging + show_status + ;; + "status" ) + category $* + cd $CLUSTER_DIR + show_status + ;; + "down" ) + category $* + # Enable the following for debugging + show_status + cd $CLUSTER_DIR + echo OVERRIDE_ENV="$ENV_FILE" docker-compose $CMD + OVERRIDE_ENV="$ENV_FILE" docker-compose $CMD + ;; + "*" ) + category $* + cd $CLUSTER_DIR + OVERRIDE_ENV="$ENV_FILE" docker-compose $CMD + ;; +esac diff --git a/integration-tests-ex/cases/cluster/BatchIndex/docker-compose.yaml b/integration-tests-ex/cases/cluster/BatchIndex/docker-compose.yaml new file mode 100644 index 00000000000..f8235db6255 --- /dev/null +++ b/integration-tests-ex/cases/cluster/BatchIndex/docker-compose.yaml @@ -0,0 +1,98 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +networks: + druid-it-net: + name: druid-it-net + ipam: + config: + - subnet: 172.172.172.0/24 + +services: + zookeeper: + extends: + file: ../Common/dependencies.yaml + service: zookeeper + + metadata: + extends: + file: ../Common/dependencies.yaml + service: metadata + + coordinator: + extends: + file: ../Common/druid.yaml + service: coordinator + container_name: coordinator + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + # The frequency with which the coordinator polls the database + # for changes. The DB population code has to wait at least this + # long for the coordinator to notice changes. + - druid_manager_segments_pollDuration=PT5S + - druid_coordinator_period=PT10S + depends_on: + - zookeeper + - metadata + + overlord: + extends: + file: ../Common/druid.yaml + service: overlord + container_name: overlord + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + depends_on: + - zookeeper + - metadata + + broker: + extends: + file: ../Common/druid.yaml + service: broker + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + depends_on: + - zookeeper + + router: + extends: + file: ../Common/druid.yaml + service: router + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + depends_on: + - zookeeper + + historical: + extends: + file: ../Common/druid.yaml + service: historical + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + depends_on: + - zookeeper + + indexer: + extends: + file: ../Common/druid.yaml + service: indexer + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + volumes: + # Test data + - ../../resources:/resources + depends_on: + - zookeeper diff --git a/integration-tests-ex/cases/cluster/Common/dependencies.yaml b/integration-tests-ex/cases/cluster/Common/dependencies.yaml new file mode 100644 index 00000000000..ccfce630a2a --- /dev/null +++ b/integration-tests-ex/cases/cluster/Common/dependencies.yaml @@ -0,0 +1,166 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# Service definitions for Druid's dependencies: ZooKeeper, MySQL, and Kafka. +# All tests need ZK and MySQL, ingestion tasks may need Kafka. +# +# These services use "official" images from the project or other sources. +# Some amount of fiddling is needed to map them into configuration which +# Druid requires. + +services: + + # Uses the official Zookeeper image + # See https://hub.docker.com/_/zookeeper + zookeeper: + image: zookeeper:${ZK_VERSION} + container_name: zookeeper + networks: + druid-it-net: + ipv4_address: 172.172.172.4 + ports: + - 2181:2181 + volumes: + - ${SHARED_DIR}/logs:/logs + environment: + ZOO_LOG4J_PROP: INFO,ROLLINGFILE + + # Uses the Bitnami Kafka image + # See https://hub.docker.com/r/bitnami/kafka/ + kafka: + image: bitnami/kafka:${KAFKA_VERSION} + container_name: kafka + ports: + - 9092:9092 + - 9093:9093 + networks: + druid-it-net: + ipv4_address: 172.172.172.2 + volumes: + - ${SHARED_DIR}/kafka:/bitnami/kafka + environment: + # This is the default: making it explicit + KAFKA_CFG_ZOOKEEPER_CONNECT: zookeeper:2181 + # Plaintext is disabled by default + ALLOW_PLAINTEXT_LISTENER: "yes" + # Adapted from base-setup.sh and Bitnami docs + KAFKA_CFG_LISTENER_SECURITY_PROTOCOL_MAP: "INTERNAL:PLAINTEXT,EXTERNAL:PLAINTEXT" + KAFKA_CFG_LISTENERS: "INTERNAL://:9092,EXTERNAL://:9093" + KAFKA_CFG_ADVERTISED_LISTENERS: "INTERNAL://kafka:9092,EXTERNAL://localhost:9093" + KAFKA_CFG_INTER_BROKER_LISTENER_NAME: INTERNAL + depends_on: + - zookeeper + + # Uses the official MySQL image + # See https://hub.docker.com/_/mysql + # The image will intialize the user and DB upon first start. + metadata: + # platform: linux/x86_64 - Add when running on M1 Macs + image: mysql:$MYSQL_IMAGE_VERSION + container_name: metadata + command: + - --character-set-server=utf8mb4 + networks: + druid-it-net: + ipv4_address: 172.172.172.3 + ports: + - 3306:3306 + volumes: + - ${SHARED_DIR}/db:/var/lib/mysql + environment: + MYSQL_ROOT_PASSWORD: driud + MYSQL_DATABASE: druid + MYSQL_USER: druid + MYSQL_PASSWORD: diurd + +## TODO: Not yet retested +### Optional supporting infra + + openldap: + image: osixia/openldap:1.4.0 + container_name: openldap + networks: + druid-it-net: + ipv4_address: 172.172.172.102 + ports: + - 8389:389 + - 8636:636 + privileged: true + volumes: + - ./ldap-configs:/container/service/slapd/assets/config/bootstrap/ldif/custom + command: --copy-service + + schema-registry: + image: confluentinc/cp-schema-registry:5.5.1 + container_name: schema-registry + ports: + - 8085:8085 + networks: + druid-it-net: + ipv4_address: 172.172.172.103 + volumes: + - ./schema-registry/jaas_config.file:/usr/lib/druid/conf/jaas_config.file + - ./schema-registry/password-file:/usr/lib/druid/conf/password-file + privileged: true + environment: + SCHEMA_REGISTRY_HOST_NAME: schema-registry + SCHEMA_REGISTRY_LISTENERS: "http://0.0.0.0:8085" + SCHEMA_REGISTRY_KAFKASTORE_BOOTSTRAP_SERVERS: kafka:9092 + SCHEMA_REGISTRY_AUTHENTICATION_METHOD: BASIC + SCHEMA_REGISTRY_AUTHENTICATION_REALM: druid + SCHEMA_REGISTRY_AUTHENTICATION_ROLES: users + SCHEMA_REGISTRY_OPTS: -Djava.security.auth.login.config=/usr/lib/druid/conf/jaas_config.file -Xmx32m + + druid-it-hadoop: + ## Giving fake version + image: druid-it/hadoop:9.9.9 + container_name: druid-it-hadoop + ports: + - 2049:2049 + - 2122:2122 + - 8020:8020 + - 8021:8021 + - 8030:8030 + - 8031:8031 + - 8032:8032 + - 8033:8033 + - 8040:8040 + - 8042:8042 + - 8088:8088 + - 8443:8443 + - 9000:9000 + - 10020:10020 + - 19888:19888 + - 34455:34455 + - 50010:50010 + - 50020:50020 + - 50030:50030 + - 50060:50060 + - 50070:50070 + - 50075:50075 + - 50090:50090 + - 51111:51111 + networks: + druid-it-net: + ipv4_address: 172.172.172.101 + privileged: true + volumes: + - ${HOME}/shared:/shared + - ./../src/test/resources:/resources + hostname: "druid-it-hadoop" + command: "bash -c 'echo Start druid-it-hadoop container... && \ + /etc/bootstrap.sh && \ + tail -f /dev/null'" + diff --git a/integration-tests-ex/cases/cluster/Common/druid.yaml b/integration-tests-ex/cases/cluster/Common/druid.yaml new file mode 100644 index 00000000000..bd5caad2232 --- /dev/null +++ b/integration-tests-ex/cases/cluster/Common/druid.yaml @@ -0,0 +1,177 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +version: "2.2" + +# IP address ranges: +# 172.172.172.2: zookeeper + kafka (dockerfile depends on this) +# 172.172.172.3: metadata store +# 172.172.172.10-19: overlord +# 172.172.172.20-29: coordinator +# 172.172.172.30-39: historicals +# 172.172.172.40-49: middle managers +# 172.172.172.50-59: indexers +# 172.172.172.60-69: brokers +# 172.172.172.70-79: routers +# 172.172.172.101+: hadoop, other supporting infra +# +# See https://docs.docker.com/compose/networking/ +# Key things to know: +# * The containers communicate using the Compose network. +# * Containers use service name as the host, container port as port +# * Ports are also exposed to the host: "host:container", use the +# host port to connect from the local machine (outside of a container). +# +# TODO: The static IPs should not be required. Remove? + +networks: + druid-it-net: + name: druid-it-net + ipam: + config: + - subnet: 172.172.172.0/24 + +services: + + overlord: + image: ${DRUID_IT_IMAGE_NAME} + container_name: overlord + networks: + druid-it-net: + ipv4_address: 172.172.172.10 + ports: + - 8090:8090 + - 8290:8290 + - 5009:8000 + volumes: + - ${SHARED_DIR}:/shared + env_file: + - environment-configs/common.env + - environment-configs/overlord.env + - ${OVERRIDE_ENV} + + coordinator: + image: ${DRUID_IT_IMAGE_NAME} + container_name: coordinator + networks: + druid-it-net: + ipv4_address: 172.172.172.20 + ports: + - 8081:8081 + - 8281:8281 + - 5006:8000 + volumes: + - ${SHARED_DIR}:/shared + env_file: + - environment-configs/common.env + - environment-configs/coordinator.env + - ${OVERRIDE_ENV} + + historical: + image: ${DRUID_IT_IMAGE_NAME} + container_name: historical + networks: + druid-it-net: + ipv4_address: 172.172.172.30 + ports: + - 8083:8083 + - 8283:8283 + - 5007:8000 + volumes: + - ${SHARED_DIR}:/shared + env_file: + - environment-configs/common.env + - environment-configs/historical.env + - ${OVERRIDE_ENV} + + middlemanager: + image: ${DRUID_IT_IMAGE_NAME} + container_name: middlemanager + networks: + druid-it-net: + ipv4_address: 172.172.172.40 + ports: + - 5008:8000 + - 8091:8091 + - 8291:8291 + - 8100:8100 + - 8101:8101 + - 8102:8102 + - 8103:8103 + - 8104:8104 + - 8105:8105 + - 8300:8300 + - 8301:8301 + - 8302:8302 + - 8303:8303 + - 8304:8304 + - 8305:8305 + volumes: + - ${SHARED_DIR}:/shared + env_file: + - environment-configs/common.env + - environment-configs/middlemanager.env + - ${OVERRIDE_ENV} + + indexer: + image: ${DRUID_IT_IMAGE_NAME} + container_name: indexer + networks: + druid-it-net: + ipv4_address: 172.172.172.50 + ports: + - 5008:8000 + - 8091:8091 + - 8291:8291 + volumes: + - ${SHARED_DIR}:/shared + env_file: + - environment-configs/common.env + - environment-configs/indexer.env + - ${OVERRIDE_ENV} + + broker: + image: ${DRUID_IT_IMAGE_NAME} + container_name: broker + networks: + druid-it-net: + ipv4_address: 172.172.172.60 + ports: + - 5005:8000 + - 8082:8082 + - 8282:8282 + volumes: + - ${SHARED_DIR}:/shared + env_file: + - environment-configs/common.env + - environment-configs/broker.env + - ${OVERRIDE_ENV} + + router: + image: ${DRUID_IT_IMAGE_NAME} + container_name: router + networks: + druid-it-net: + ipv4_address: 172.172.172.70 + ports: + - 5004:8000 + - 8888:8888 + - 9088:9088 + volumes: + - ${SHARED_DIR}:/shared + env_file: + - environment-configs/common.env + - environment-configs/router.env + - ${OVERRIDE_ENV} diff --git a/integration-tests-ex/cases/cluster/Common/environment-configs/auth.env b/integration-tests-ex/cases/cluster/Common/environment-configs/auth.env new file mode 100644 index 00000000000..fec58cc0a00 --- /dev/null +++ b/integration-tests-ex/cases/cluster/Common/environment-configs/auth.env @@ -0,0 +1,53 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +# Settings for a secure cluster. +# Disabled by default for each of testing. +# Use only for security tests. + +druid_auth_authenticator_basic_authorizerName=basic +druid_auth_authenticator_basic_initialAdminPassword=priest +druid_auth_authenticator_basic_initialInternalClientPassword=warlock +druid_auth_authenticator_basic_type=basic +druid_auth_authenticatorChain=["basic"] +druid_auth_authorizer_basic_type=basic +druid_auth_authorizers=["basic"] +druid_client_https_certAlias=druid +druid_client_https_keyManagerPassword=druid123 +druid_client_https_keyStorePassword=druid123 +druid_client_https_keyStorePath=/tls/server.p12 +druid_client_https_protocol=TLSv1.2 +druid_client_https_trustStoreAlgorithm=PKIX +druid_client_https_trustStorePassword=druid123 +druid_client_https_trustStorePath=/tls/truststore.jks +druid_enableTlsPort=true +druid_server_http_allowedHttpMethods=["OPTIONS"] +druid_server_https_certAlias=druid +druid_server_https_keyManagerPassword=druid123 +druid_server_https_keyStorePassword=druid123 +druid_server_https_keyStorePath=/tls/server.p12 +druid_server_https_keyStoreType=PKCS12 +druid_server_https_requireClientCertificate=true +druid_server_https_trustStoreAlgorithm=PKIX +druid_server_https_trustStorePassword=druid123 +druid_server_https_trustStorePath=/tls/truststore.jks +druid_server_https_validateHostnames=true +druid_escalator_authorizerName=basic +druid_escalator_internalClientPassword=warlock +druid_escalator_internalClientUsername=druid_system +druid_escalator_type=basic +druid_server_https_crlPath=/tls/revocations.crl diff --git a/integration-tests-ex/cases/cluster/Common/environment-configs/broker.env b/integration-tests-ex/cases/cluster/Common/environment-configs/broker.env new file mode 100644 index 00000000000..652bbaf69df --- /dev/null +++ b/integration-tests-ex/cases/cluster/Common/environment-configs/broker.env @@ -0,0 +1,53 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +DRUID_SERVICE=broker + +# JAVA OPTS +SERVICE_DRUID_JAVA_OPTS=-Xms192m -Xmx256m + +# Druid configs +druid_host=broker + +# HTTP server settings +druid_server_http_numThreads=40 +druid_server_https_crlPath=/tls/revocations.crl + +# HTTP client settings +druid_broker_http_numConnections=20 +druid_broker_http_readTimeout=PT5M + +# Processing threads and buffers +druid_processing_buffer_sizeBytes=25000000 +druid_processing_numMergeBuffers=2 +druid_processing_numThreads=1 +druid_processing_tmpDir=/tmp/druid/processing + +# Query cache +druid_broker_cache_useCache=true +druid_broker_cache_populateCache=true +druid_cache_sizeInBytes=40000000 + +druid_query_groupBy_maxOnDiskStorage=300000000 +druid_lookup_namespace_cache_type=onHeap +druid_auth_basic_common_cacheDirectory=/tmp/authCache/broker +druid_sql_avatica_enable=true +druid_query_scheduler_laning_strategy=manual +druid_query_scheduler_laning_lanes_one=1 +druid_segmentCache_locations=[{"path":"/shared/druid/brokerIndexCache","maxSize":1000000000}] +druid_server_maxSize=1000000000 +druid_sql_planner_metadataRefreshPeriod=PT15S diff --git a/integration-tests-ex/cases/cluster/Common/environment-configs/common.env b/integration-tests-ex/cases/cluster/Common/environment-configs/common.env new file mode 100644 index 00000000000..350a1f5b792 --- /dev/null +++ b/integration-tests-ex/cases/cluster/Common/environment-configs/common.env @@ -0,0 +1,117 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +LANG=C.UTF-8 +LANGUAGE=C.UTF-8 +LC_ALL=C.UTF-8 + +# JAVA OPTS +# -XX:HeapDumpPath set in container based on the service and intance +DRUID_COMMON_JAVA_OPTS=-server -Duser.timezone=UTC -Dfile.encoding=UTF-8 -XX:+UseG1GC -XX:+ExitOnOutOfMemoryError -XX:+HeapDumpOnOutOfMemoryError +DRUID_SERVICE_JAVA_OPTS= + +# Debugging. Enabled by default. Always uses port 8000 in the container. +# May to unique ports onto the host in Docker Compose. +DEBUG_PORT= +DEBUG_OPTS=-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=8000 + +# Extra classpath. Standard classpath includes Druid libs, Hadoop config, Druid config. +DRUID_CLASSPATH= + +# Number of an instance when there are two or more of the same type. +DRUID_INSTANCE= + +# Druid configs +# Mostly adapted from the micro-quickstart/_common file. + +# Hostname +# druid.host is set on each host by the launch script + +# Extensions specified in the load list will be loaded by Druid at runtime. +# The extension jars must be installed as part of Druid, or via the image +# build script. +# +# The launch script creates druid_extensions_loadList by combining two test-specific +# variables: druid_standard_loadList defined here, and druid_test_loadList, defined +# in a docker-compose.yaml file, for any test-specific extensions. +# See compose.md for more details. +druid_standard_loadList=mysql-metadata-storage,it-tools,druid-lookups-cached-global,druid-histogram,druid-datasketches,druid-parquet-extensions,druid-avro-extensions,druid-protobuf-extensions,druid-orc-extensions,druid-kafka-indexing-service,druid-s3-extensions + +# Location of Hadoop dependencies provided at runtime in the shared directory. +druid_extensions_hadoopDependenciesDir=/shared/hadoop-dependencies + +# Logging +druid_startup_logging_logProperties=true + +# Zookeeper +# Name of the ZK container, mapped to a host name +druid_zk_service_host=zookeeper +druid_zk_paths_base=/druid + +# Metadata storage +# For MySQL (MySQL JDBC installed in $DRUID_HOME/lib) +druid_metadata_storage_type=mysql +druid_metadata_storage_connector_connectURI=jdbc:mysql://metadata/druid +druid_metadata_storage_connector_user=druid +druid_metadata_storage_connector_password=diurd + +# Deep storage +druid_storage_type=local +druid_storage_storageDirectory=/shared/druid/storage + +# Indexing service logs +druid_indexer_logs_type=file +druid_indexer_logs_directory=/shared/tasklogs + +# Service discovery +druid_selectors_indexing_serviceName=druid/overlord +druid_selectors_coordinator_serviceName=druid/coordinator + +# Monitoring +druid_monitoring_monitors=["org.apache.druid.java.util.metrics.JvmMonitor"] +druid_emitter=noop +druid_emitter_logging_logLevel=info + +# Storage type of double columns +druid_indexing_doubleStorage=double + +# SQL +druid_sql_enable=true +druid_sql_planner_authorizeSystemTablesDirectly=true + +# Lookups +druid_lookup_numLookupLoadingThreads=1 + +# Test-specific +druid_server_http_numThreads=20 +# Allow OPTIONS method for ITBasicAuthConfigurationTest.testSystemSchemaAccess +druid_auth_basic_common_maxSyncRetries=20 +druid_request_logging_type=slf4j + +# TODO: Can these 4 move to the Coordinator config? +druid_coordinator_kill_supervisor_on=true +druid_coordinator_kill_supervisor_period=PT10S +druid_coordinator_kill_supervisor_durationToRetain=PT0M +druid_coordinator_period_metadataStoreManagementPeriod=PT10S + +# TODO: Can the following be moved to Overlord? Or removed? +# Testing the legacy config from https://github.com/apache/druid/pull/10267 +# Can remove this when the flag is no longer needed +druid_indexer_task_ignoreTimestampSpecForDruidInputSource=true + + +# TODO: Pass this from the test (AzureDeepStorage) diff --git a/integration-tests-ex/cases/cluster/Common/environment-configs/coordinator.env b/integration-tests-ex/cases/cluster/Common/environment-configs/coordinator.env new file mode 100644 index 00000000000..fa20cf6ccaf --- /dev/null +++ b/integration-tests-ex/cases/cluster/Common/environment-configs/coordinator.env @@ -0,0 +1,39 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +DRUID_SERVICE=coordinator + +# JAVA OPTS +DRUID_SERVICE_JAVA_OPTS=-Xmx64m -Xms64m + +# Druid configs +druid_host=coordinator +druid_coordinator_startDelay=PT5S +druid_manager_lookups_hostUpdateTimeout=PT30S +druid_manager_lookups_period=10000 +druid_manager_lookups_threadPoolSize=2 +druid_manager_config_pollDuration=PT10S +druid_manager_rules_pollDuration=PT10S +druid_manager_segments_pollDuration=PT2S +druid_auth_basic_common_cacheDirectory=/tmp/authCache/coordinator +druid_auth_unsecuredPaths=["/druid/coordinator/v1/loadqueue"] +druid_server_https_crlPath=/tls/revocations.crl +druid_coordinator_period_indexingPeriod=PT180000S + +# 2x indexing period so that kill period is valid +druid_coordinator_kill_period=PT360000S +druid_coordinator_period=PT1S diff --git a/integration-tests-ex/cases/cluster/Common/environment-configs/historical.env b/integration-tests-ex/cases/cluster/Common/environment-configs/historical.env new file mode 100644 index 00000000000..cf969f62894 --- /dev/null +++ b/integration-tests-ex/cases/cluster/Common/environment-configs/historical.env @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +DRUID_SERVICE=historical + +# JAVA OPTS +SERVICE_DRUID_JAVA_OPTS=-Xmx512m -Xms512m + +# Druid configs +druid_host=historical +druid_processing_buffer_sizeBytes=25000000 +druid_processing_numThreads=2 +druid_query_groupBy_maxOnDiskStorage=300000000 +druid_segmentCache_locations=[{"path":"/shared/druid/indexCache","maxSize":5000000000}] +druid_auth_basic_common_cacheDirectory=/tmp/authCache/historical +druid_server_https_crlPath=/tls/revocations.crl diff --git a/integration-tests-ex/cases/cluster/Common/environment-configs/indexer.env b/integration-tests-ex/cases/cluster/Common/environment-configs/indexer.env new file mode 100644 index 00000000000..1c6afffb182 --- /dev/null +++ b/integration-tests-ex/cases/cluster/Common/environment-configs/indexer.env @@ -0,0 +1,35 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +DRUID_SERVICE=indexer + +# JAVA OPTS +SERVICE_DRUID_JAVA_OPTS=-Xmx1g -Xms512m + +# Druid configs +druid_host=indexer +druid_server_http_numThreads=4 +druid_storage_storageDirectory=/shared/storage + +druid_processing_buffer_sizeBytes=25000000 +druid_processing_numThreads=1 +druid_selectors_indexing_serviceName=druid/overlord +druid_indexer_task_chathandler_type=announce +druid_auth_basic_common_cacheDirectory=/tmp/authCache/indexer +druid_startup_logging_logProperties=true +druid_server_https_crlPath=/tls/revocations.crl +druid_worker_capacity=10 diff --git a/integration-tests-ex/cases/cluster/Common/environment-configs/middlemanager.env b/integration-tests-ex/cases/cluster/Common/environment-configs/middlemanager.env new file mode 100644 index 00000000000..8ea21dac6e2 --- /dev/null +++ b/integration-tests-ex/cases/cluster/Common/environment-configs/middlemanager.env @@ -0,0 +1,37 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +DRUID_SERVICE=middleManager + +# JAVA OPTS +SERVICE_DRUID_JAVA_OPTS=-Xmx64m -Xms64m + +# Druid configs +druid_host=middlemanager +druid_server_http_numThreads=100 +druid_storage_storageDirectory=/shared/storage +druid_indexer_runner_javaOptsArray=["-server", "-Xmx256m", "-Xms256m", "-XX:NewSize=128m", "-XX:MaxNewSize=128m", "-XX:+UseG1GC", "-Duser.timezone=UTC", "-Dfile.encoding=UTF-8", "-Dlog4j.configurationFile=/shared/resources/log4j2.xml"] + +druid_indexer_fork_property_druid_processing_buffer_sizeBytes=25000000 +druid_indexer_fork_property_druid_processing_numThreads=1 +druid_indexer_fork_server_http_numThreads=20 +druid_selectors_indexing_serviceName=druid/overlord +druid_indexer_task_chathandler_type=announce +druid_auth_basic_common_cacheDirectory=/tmp/authCache/middleManager +druid_startup_logging_logProperties=true +druid_server_https_crlPath=/tls/revocations.crl +druid_worker_capacity=10 diff --git a/integration-tests-ex/cases/cluster/Common/environment-configs/overlord.env b/integration-tests-ex/cases/cluster/Common/environment-configs/overlord.env new file mode 100644 index 00000000000..1e73199ee58 --- /dev/null +++ b/integration-tests-ex/cases/cluster/Common/environment-configs/overlord.env @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +DRUID_SERVICE=overlord + +# JAVA OPTS +SERVICE_DRUID_JAVA_OPTS=-Xmx64m -Xms64m + +# Druid configs +druid_host=overlord + +druid_indexer_queue_startDelay=PT5S +druid_indexer_storage_type=metadata +druid_indexer_runner_type=remote +druid_auth_basic_common_cacheDirectory=/tmp/authCache/overlord +druid_server_https_crlPath=/tls/revocations.crl diff --git a/integration-tests-ex/cases/cluster/Common/environment-configs/router.env b/integration-tests-ex/cases/cluster/Common/environment-configs/router.env new file mode 100644 index 00000000000..2e8c5bc6f43 --- /dev/null +++ b/integration-tests-ex/cases/cluster/Common/environment-configs/router.env @@ -0,0 +1,37 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +# + +DRUID_SERVICE=router + +# JAVA OPTS +DRUID_SERVICE_JAVA_OPTS=-Xmx64m -Xms64m + +# Druid configs +druid_host=router +druid_auth_basic_common_cacheDirectory=/tmp/authCache/router +druid_sql_avatica_enable=true + +# HTTP proxy +druid_router_http_numConnections=50 +druid_router_http_readTimeout=PT5M +druid_router_http_numMaxThreads=100 +druid_server_http_numThreads=100 + +# Management proxy to coordinator / overlord: required for unified web console. +druid_router_managementProxy_enabled=true diff --git a/integration-tests-ex/cases/cluster/HighAvailability/docker-compose.yaml b/integration-tests-ex/cases/cluster/HighAvailability/docker-compose.yaml new file mode 100644 index 00000000000..bcecf4d9eee --- /dev/null +++ b/integration-tests-ex/cases/cluster/HighAvailability/docker-compose.yaml @@ -0,0 +1,157 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +networks: + druid-it-net: + name: druid-it-net + ipam: + config: + - subnet: 172.172.172.0/24 + +services: + zookeeper: + extends: + file: ../Common/dependencies.yaml + service: zookeeper + + metadata: + extends: + file: ../Common/dependencies.yaml + service: metadata + + coordinator-one: + extends: + file: ../Common/druid.yaml + service: coordinator + container_name: coordinator-one + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + - DRUID_INSTANCE=one + # The frequency with which the coordinator polls the database + # for changes. The DB population code has to wait at least this + # long for the coordinator to notice changes. + - druid_manager_segments_pollDuration=PT5S + - druid_coordinator_period=PT10S + - druid_host=coordinator-one + depends_on: + - zookeeper + - metadata + + # The second Coordinator (and Overlord) cannot extend + # The base service: they need distinct ports. + coordinator-two: + image: ${DRUID_IT_IMAGE_NAME} + container_name: coordinator-two + networks: + druid-it-net: + ipv4_address: 172.172.172.120 + ports: + - 18081:8081 + - 18281:8281 + - 15006:8000 + volumes: + - ${SHARED_DIR}:/shared + env_file: + - ../Common/environment-configs/common.env + - ../Common/environment-configs/coordinator.env + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + - DRUID_INSTANCE=two + - druid_manager_segments_pollDuration=PT5S + - druid_coordinator_period=PT10S + - druid_host=coordinator-two + depends_on: + - zookeeper + - metadata + + overlord-one: + extends: + file: ../Common/druid.yaml + service: overlord + container_name: overlord-one + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + - DRUID_INSTANCE=one + - druid_host=overlord-one + depends_on: + - zookeeper + - metadata + + overlord-two: + image: ${DRUID_IT_IMAGE_NAME} + container_name: overlord-two + networks: + druid-it-net: + ipv4_address: 172.172.172.110 + ports: + - 18090:8090 + - 18290:8290 + - 15009:8000 + volumes: + - ${SHARED_DIR}:/shared + env_file: + - ../Common/environment-configs/common.env + - ../Common/environment-configs/overlord.env + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + - DRUID_INSTANCE=two + - druid_host=overlord-two + depends_on: + - zookeeper + - metadata + + broker: + extends: + file: ../Common/druid.yaml + service: broker + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + depends_on: + - zookeeper + + router: + extends: + file: ../Common/druid.yaml + service: router + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + depends_on: + - zookeeper + + # The custom node role has no base definition. Also, there is + # no environment file: the needed environment settings are + # given here. + custom-node-role: + image: ${DRUID_IT_IMAGE_NAME} + container_name: custom-node-role + networks: + druid-it-net: + ipv4_address: 172.172.172.90 + ports: + - 50011:50011 + - 9301:9301 + - 9501:9501 + - 5010:8000 + volumes: + - ${SHARED_DIR}:/shared + env_file: + - ../Common/environment-configs/common.env + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + - DRUID_SERVICE=custom-node-role + - SERVICE_DRUID_JAVA_OPTS=-Xmx64m -Xms64m + - druid_host=custom-node-role + - druid_auth_basic_common_cacheDirectory=/tmp/authCache/custom_node_role + - druid_server_https_crlPath=/tls/revocations.crl diff --git a/integration-tests-ex/cases/pom.xml b/integration-tests-ex/cases/pom.xml new file mode 100644 index 00000000000..5c34af881d6 --- /dev/null +++ b/integration-tests-ex/cases/pom.xml @@ -0,0 +1,374 @@ + + + + + 4.0.0 + + org.apache.druid.integration-tests + druid-it-cases + druid-it-cases + New Integration Tests + + + org.apache.druid + druid + 0.24.0-SNAPSHOT + ../../pom.xml + + + + MustSetACategory + + + + + org.apache.druid + druid-integration-tests + ${project.parent.version} + + + + org.apache.druid + druid-integration-tests + ${project.parent.version} + tests + test-jar + test + + + org.apache.druid + druid-core + ${project.parent.version} + + + org.apache.druid + druid-processing + ${project.parent.version} + + + org.apache.druid + druid-server + ${project.parent.version} + + + org.apache.druid + druid-services + ${project.parent.version} + + + org.apache.druid + druid-indexing-service + ${project.parent.version} + + + com.google.inject + guice + + + com.google.inject.extensions + guice-multibindings + + + org.apache.curator + curator-framework + + + org.apache.curator + curator-client + + + com.google.guava + guava + + + com.fasterxml.jackson.dataformat + jackson-dataformat-yaml + + + com.fasterxml.jackson.core + jackson-annotations + + + com.fasterxml.jackson.core + jackson-databind + + + com.fasterxml.jackson.core + jackson-core + + + joda-time + joda-time + + + commons-io + commons-io + + + com.github.docker-java + docker-java-core + 3.2.0 + + + com.github.docker-java + docker-java-api + 3.2.0 + + + com.github.docker-java + docker-java-transport-netty + 3.2.11 + + + com.github.docker-java + docker-java + 3.2.0 + + + com.github.docker-java + docker-java-transport-jersey + + + io.netty + netty-transport-native-kqueue + + + + + mysql + mysql-connector-java + ${mysql.version} + runtime + + + org.jdbi + jdbi + + + org.apache.druid.extensions + mysql-metadata-storage + ${project.parent.version} + + + org.apache.druid.extensions + druid-azure-extensions + ${project.parent.version} + provided + + + org.apache.druid.extensions + druid-hdfs-storage + ${project.parent.version} + provided + + + com.amazonaws + aws-java-sdk-bundle + + + + + org.apache.commons + commons-lang3 + + + io.netty + netty + + + org.apache.kafka + kafka-clients + ${apache.kafka.version} + test + + + com.google.code.findbugs + jsr305 + + + junit + junit + test + + + pl.pragmatists + JUnitParams + test + + + + + + + + org.apache.maven.plugins + maven-surefire-plugin + + + **/IT* + + + + + org.apache.maven.plugins + maven-dependency-plugin + + + + org.glassfish.hk2.external:jakarta.inject + + + + mysql:mysql-connector-java:jar + + + + + + + + + IT-HighAvailability + + false + + + HighAvailability + + + + IT-BatchIndex + + false + + + BatchIndex + + + + IT-InputFormat + + false + + + InputFormat + + + + IT-AzureDeepStorage + + false + + + AzureDeepStorage + + + + docker-tests + + false + + + + + maven-failsafe-plugin + 3.0.0-M7 + + + + org.apache.maven.surefire + surefire-junit47 + 3.0.0-M7 + + + + + functional-test + integration-test + + integration-test + + + + true + + org.apache.druid.testsEx.categories.${it.category} + + + + verify + verify + + verify + + + + + + + org.codehaus.mojo + exec-maven-plugin + + + start-cluster + pre-integration-test + + exec + + + ${project.basedir} + bash + + cluster.sh + up + ${it.category} + + + + + + stop-cluster + post-integration-test + + exec + + + ${project.basedir} + bash + + cluster.sh + down + ${it.category} + + + + + + + + + + diff --git a/integration-tests-ex/cases/resources/README.md b/integration-tests-ex/cases/resources/README.md new file mode 100644 index 00000000000..7ae95d8a55f --- /dev/null +++ b/integration-tests-ex/cases/resources/README.md @@ -0,0 +1,42 @@ + + +# Test Data + +This directory contains resources, such as test data, mounted into +Druid indexer container at `/resources`. +That name is chosen for backward compatibility with the previous IT +version that mounted `/src/test/resources` to that mount point. +See the `indexer` node in `docker-compose.yaml` for `BatchIndex` +for the actual volume mount. + +The many indexer specs expect data to be at +`/resources/data/batch_index/json/spec.json`, where `spec` is the +name of some ingest spec. The structure here was chosen to allow +the existing specs to be reused without change. + +Put data for the Indexer in this folder. Put files to be used by +tests in `/src/test/resources`. That way, we only mount into the container +the data which the server needs. Resources which tests need, such as +queries, specs and other resources, are on the class path of the test, +but are not visible to the server. + +Paths within this folder are the same as the former +`/src/test/resources` folder so that the many indexer specs don't +have to change. diff --git a/integration-tests-ex/cases/resources/data/batch_index/avro/wikipedia_index_data1.avro b/integration-tests-ex/cases/resources/data/batch_index/avro/wikipedia_index_data1.avro new file mode 100644 index 00000000000..5ed0d607035 Binary files /dev/null and b/integration-tests-ex/cases/resources/data/batch_index/avro/wikipedia_index_data1.avro differ diff --git a/integration-tests-ex/cases/resources/data/batch_index/avro/wikipedia_index_data2.avro b/integration-tests-ex/cases/resources/data/batch_index/avro/wikipedia_index_data2.avro new file mode 100644 index 00000000000..aa25803d5d2 Binary files /dev/null and b/integration-tests-ex/cases/resources/data/batch_index/avro/wikipedia_index_data2.avro differ diff --git a/integration-tests-ex/cases/resources/data/batch_index/avro/wikipedia_index_data3.avro b/integration-tests-ex/cases/resources/data/batch_index/avro/wikipedia_index_data3.avro new file mode 100644 index 00000000000..2e0f9f2e1d0 Binary files /dev/null and b/integration-tests-ex/cases/resources/data/batch_index/avro/wikipedia_index_data3.avro differ diff --git a/integration-tests-ex/cases/resources/data/batch_index/csv/wikipedia_index_data1.csv b/integration-tests-ex/cases/resources/data/batch_index/csv/wikipedia_index_data1.csv new file mode 100644 index 00000000000..c2e2b0714d8 --- /dev/null +++ b/integration-tests-ex/cases/resources/data/batch_index/csv/wikipedia_index_data1.csv @@ -0,0 +1,4 @@ +timestamp,page,language,user,unpatrolled,newPage,robot,anonymous,namespace,continent,country,region,city,added,deleted,delta +2013-08-31T01:02:33Z,Gypsy Danger,en,nuclear,TRUE,TRUE,FALSE,FALSE,article,North America,United States,Bay Area,San Francisco,57,200,-143 +2013-08-31T03:32:45Z,Striker Eureka,en,speed,FALSE,TRUE,TRUE,FALSE,wikipedia,Australia,Australia,Cantebury,Syndey,459,129,330 +2013-08-31T07:11:21Z,Cherno Alpha,ru,masterYi,FALSE,TRUE,TRUE,FALSE,article,Asia,Russia,Oblast,Moscow,123,12,111 diff --git a/integration-tests-ex/cases/resources/data/batch_index/csv/wikipedia_index_data2.csv b/integration-tests-ex/cases/resources/data/batch_index/csv/wikipedia_index_data2.csv new file mode 100644 index 00000000000..cbed2815db9 --- /dev/null +++ b/integration-tests-ex/cases/resources/data/batch_index/csv/wikipedia_index_data2.csv @@ -0,0 +1,4 @@ +timestamp,page,language,user,unpatrolled,newPage,robot,anonymous,namespace,continent,country,region,city,added,deleted,delta +2013-08-31T11:58:39Z,Crimson Typhoon,zh,triplets,TRUE,FALSE,TRUE,FALSE,wikipedia,Asia,China,Shanxi,Taiyuan,905,5,900 +2013-08-31T12:41:27Z,Coyote Tango,ja,stringer,TRUE,FALSE,TRUE,FALSE,wikipedia,Asia,Japan,Kanto,Tokyo,1,10,-9 +2013-09-01T01:02:33Z,Gypsy Danger,en,nuclear,TRUE,TRUE,FALSE,FALSE,article,North America,United States,Bay Area,San Francisco,57,200,-143 diff --git a/integration-tests-ex/cases/resources/data/batch_index/csv/wikipedia_index_data3.csv b/integration-tests-ex/cases/resources/data/batch_index/csv/wikipedia_index_data3.csv new file mode 100644 index 00000000000..51d6d21d0d2 --- /dev/null +++ b/integration-tests-ex/cases/resources/data/batch_index/csv/wikipedia_index_data3.csv @@ -0,0 +1,5 @@ +timestamp,page,language,user,unpatrolled,newPage,robot,anonymous,namespace,continent,country,region,city,added,deleted,delta +2013-09-01T03:32:45Z,Striker Eureka,en,speed,FALSE,TRUE,TRUE,FALSE,wikipedia,Australia,Australia,Cantebury,Syndey,459,129,330 +2013-09-01T07:11:21Z,Cherno Alpha,ru,masterYi,FALSE,TRUE,TRUE,FALSE,article,Asia,Russia,Oblast,Moscow,123,12,111 +2013-09-01T11:58:39Z,Crimson Typhoon,zh,triplets,TRUE,FALSE,TRUE,FALSE,wikipedia,Asia,China,Shanxi,Taiyuan,905,5,900 +2013-09-01T12:41:27Z,Coyote Tango,ja,stringer,TRUE,FALSE,TRUE,FALSE,wikipedia,Asia,Japan,Kanto,Tokyo,1,10,-9 diff --git a/integration-tests-ex/cases/resources/data/batch_index/hadoop_tsv/batch_hadoop.data b/integration-tests-ex/cases/resources/data/batch_index/hadoop_tsv/batch_hadoop.data new file mode 100644 index 00000000000..5cbad1d1a87 --- /dev/null +++ b/integration-tests-ex/cases/resources/data/batch_index/hadoop_tsv/batch_hadoop.data @@ -0,0 +1,3360 @@ +2014102000 location_2 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/EjCpo+GRSVI= AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102000 location_4 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/f9W8+ncS4gY= AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102000 location_5 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/GgfpDdmxLhY= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102000 location_4 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/K8REsye3fwU= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102000 location_2 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/WsCXYZhyozE= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102000 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/dK4342O/FTk= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102000 location_5 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/AlCrGy8+2nQ= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102000 location_1 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/UI5wPt/62mg= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102000 location_3 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/MOmSBXAdjjM= AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102000 location_4 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/knV/W1AAhkI= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102001 location_5 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/eRHUCoYK01Y= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102001 location_2 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/kQuDnt2wKl0= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102001 location_2 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/at+3K9VLgDA= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102001 location_1 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/MP+fZwNS/C8= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102001 location_3 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/hezyCab9fwk= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102001 location_2 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/mZpq/D+8WkA= AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102001 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/ai4fvSJgG3Y= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102001 location_1 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/5KOnRDI9Al8= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102001 location_4 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/2bsjx1MucyI= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102001 location_3 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/GBq+Q6KT+Sw= AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102002 location_2 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/biX5Pvg75gg= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102002 location_2 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/RnfDgoKmUyo= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102002 location_2 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/0jOeJMIdryc= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102002 location_5 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/vfoeSlxgIk8= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102002 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/YkYckSgh7Rs= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102002 location_4 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/8JnLkGK3MDQ= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102002 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/P7jPaTii00U= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102002 location_5 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/UY785K8nMmk= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102002 location_5 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/M4NSbjrLAiI= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102002 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/E8lPrdC2EAQ= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102003 location_5 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/IWfsCAHfuVc= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102003 location_4 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/3kjlykqJ4Ak= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102003 location_5 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/1NWqkl/Wh0Y= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102003 location_3 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/Wnr0C+7kAW0= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102003 location_2 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/IQ9EMf+HsUs= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102003 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9//TelFFXedRM= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102003 location_2 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/Mrd4L1nzm2U= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102003 location_3 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/rUhLLKkNgko= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102003 location_4 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/MA2BbW3p1WE= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102003 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/yHliv0IAwGA= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102004 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9//DrPhe1qSF8= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102004 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/EzLoG0F2cAY= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102004 location_2 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/m88oI8oTtGM= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102004 location_3 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/CHqdGL+V9SU= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102004 location_3 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/IADP3uPMfnA= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102004 location_2 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/Vy3QzlASAnQ= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102004 location_4 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/p/nv5UT0hkk= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102004 location_3 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/xcQkGwA6LkQ= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102004 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/PVVoyeQgLkQ= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102004 location_5 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/Jf5Hi1pRdAE= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102005 location_4 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/5T6JRK+ic3I= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102005 location_2 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/xLPzAF186Xg= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102005 location_4 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/S0bU2mFrLFc= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102005 location_4 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/3hYiQkhVxVI= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102005 location_2 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/ttepKTpirGw= AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102005 location_4 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/8lrKkjilpQg= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102005 location_3 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/gs9UAv7fG3o= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102005 location_5 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/DEC78zuV+Tc= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102005 location_5 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/cJ4RK5Cn4h0= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102005 location_1 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/2smHx2YUeWc= AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102006 location_2 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/Wnr0C+7kAW0= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102006 location_3 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/hI0eujTrADA= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102006 location_5 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/OaJYFiAa8gY= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102006 location_2 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/n/gqwVpESw0= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102006 location_1 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/tO+rpwDU21c= AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102006 location_2 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/w3fCLLXxhTM= AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102006 location_3 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/zrw+p+CnfCE= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102006 location_1 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/uUCbqGlTpks= AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102006 location_5 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/6+5YY4zs8Uc= AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102006 location_3 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/XKoYdy8CCF0= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102007 location_3 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/3FisV+hJ3yw= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102007 location_1 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/zGwavWRoO1I= AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102007 location_1 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/doRdr3CBth0= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102007 location_5 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/60YW0tKbryg= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102007 location_3 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/GqHtnaQebh8= AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102007 location_5 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/R74pFhujI1g= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102007 location_3 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/opoPGsJOGzI= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102007 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/JL0s8uipTQw= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102007 location_5 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/eoPH+uP4L2o= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102007 location_3 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/glL81aNiBEk= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102008 location_3 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/SfSvrBCAGT8= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102008 location_4 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/4OCDCLYpbU4= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102008 location_1 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/s/5gNjr5xgA= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102008 location_2 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/OLUpmjmHqCQ= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102008 location_5 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/mmb9xnNfPys= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102008 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/WrpnFRSQYUg= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102008 location_4 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/sLCb7NQp6jU= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102008 location_5 product_1 AwEDAAAAAgABAAAAAAAAAP////////9//yBBZPvR0D0= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102008 location_5 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/vwAK5Fd0oB4= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102008 location_5 product_9 AwEDAAAAAgABAAAAAAAAAP////////9//59wJDrWEVA= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102009 location_5 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/I7JGqQaXtyY= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102009 location_3 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/m654qCtd3XA= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102009 location_5 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/9Imuzu07tjo= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102009 location_4 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/lXd3l8gTBlU= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102009 location_4 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/oCHsQOWjmWA= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102009 location_5 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/bU6EPMtqWCk= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102009 location_1 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/L19+i80vKGY= AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102009 location_3 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/GoLfPgnv0h8= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102009 location_2 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/S5yGd7ik+ls= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102009 location_2 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/WrpnFRSQYUg= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102010 location_2 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/TGBWoBYd0xo= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102010 location_4 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/h2ewwHINmSk= AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102010 location_3 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/TJyqB+exnQc= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102010 location_1 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/uFHSG8MeTk0= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102010 location_3 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/gBbNQWl14yE= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102010 location_2 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/wY0a+8BvDyE= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102010 location_3 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/EIzvyk0fDAE= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102010 location_4 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/kUXocIeoQ3Y= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102010 location_1 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/mYUlG2lnaCU= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102010 location_2 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/ZHFFmMMR7m0= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102011 location_3 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/zYTnqCnaxmQ= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102011 location_3 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/ahH2H456QGA= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102011 location_1 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/1ZXjBwzc5mY= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102011 location_4 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/UmqQBatvpGU= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102011 location_2 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/IP/Y4LmEeCg= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102011 location_4 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/hEbyA8utUj0= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102011 location_5 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/qskSK5ugEzE= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102011 location_4 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/3ZD8vvNwggU= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102011 location_4 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/iz9LPZDxkzU= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102011 location_3 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/2KyOnv+ZQgA= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102012 location_2 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/S5z1m0RlNS4= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102012 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/6X2CHREsO08= AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102012 location_3 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/Ux2z7GiIZRI= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102012 location_4 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/Hd4urCIDPXM= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102012 location_2 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/rqxDRRBvSGs= AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102012 location_5 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/xw/wDgSRo2w= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102012 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/ee3Wct+FhVU= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102012 location_3 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/wIbQEqHPOnI= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102012 location_2 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/uuWAa5Mp6Xo= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102012 location_1 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/KCZEeTWPv2U= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102013 location_4 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/c2FHaM2oagw= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102013 location_3 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/FBpYL5B+D2s= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102013 location_4 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/WkP8v1HrrBE= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102013 location_3 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/k8xLUDGh7Vs= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102013 location_4 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/hTUjQbz4lQ4= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102013 location_4 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/x0fXR9FoUFc= AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102013 location_5 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/WjG6L5GPMzM= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102013 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/Up4+X3MoCQs= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102013 location_3 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/2B8H3QEtchw= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102013 location_1 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/kNbdEaLBwXc= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102014 location_3 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/zoGitFDsjhk= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102014 location_2 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/7ECGq5853HI= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102014 location_5 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/9bjw8Km9dX8= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102014 location_4 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/WBySGh7tzU4= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102014 location_4 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/pAJwBPZk7EM= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102014 location_5 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/UZ3mJRW9cnQ= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102014 location_4 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/V1hWSuT9BB4= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102014 location_3 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/AcCYNbHzWm8= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102014 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/1S/zsqYXshA= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102014 location_3 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/8ZKc81ZP52M= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102015 location_1 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/qSVdyQHpwjI= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102015 location_2 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/xTu6Dmnq6xQ= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102015 location_1 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/CX1yxvlWoBI= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102015 location_3 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/ER5jgCH5iHw= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102015 location_2 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/lNUqcsSZrTI= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102015 location_2 product_1 AwEDAAAAAgABAAAAAAAAAP////////9//59wJDrWEVA= AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102015 location_5 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/wKk8PlXz2To= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102015 location_5 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/qCn38yfeKzY= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102015 location_1 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/95liMtBPMFw= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102015 location_3 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/Auhs9UTEh2w= AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102016 location_1 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/eX44R7Sy5WY= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102016 location_2 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/9gR5U6xDKwg= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102016 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/UfC8STenaxA= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102016 location_1 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/RqI5Z2Ku0Xk= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102016 location_3 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/151P1dI3l3U= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102016 location_5 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/x3XuZ+6TSjo= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102016 location_1 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/ToEo4LH0vAg= AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102016 location_5 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/MOHYU96cf3A= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102016 location_4 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/3SZ70yZAYnU= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102016 location_4 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/ogaNqO1hUEc= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102017 location_5 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/+g7ltp9kX28= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102017 location_5 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/TQlUUHD8cig= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102017 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/CelCe4afiyw= AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102017 location_3 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/WG8Owc1482k= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102017 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/bS/gorLelUY= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102017 location_4 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/FC6Tbd/+Xhg= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102017 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/iEHM7XP21zw= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102017 location_5 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/+Ln8mKeR+kc= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102017 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/Y2lZhpTodF0= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102017 location_1 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/TIy6r9s4uRg= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102018 location_3 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/ftbXf1FxZj0= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102018 location_5 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/f2DCYGmvvXA= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102018 location_5 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/2qgXkg44iD8= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102018 location_3 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/VDrYTcwlen0= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102018 location_1 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/oATATWw1lEo= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102018 location_5 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/NC0lbYkVKF8= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102018 location_4 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/NS+5y3SlezU= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102018 location_1 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/R7EouBlgPnQ= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102018 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/0m+JD4989zU= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102018 location_5 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/XmyrLJc72Tk= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102019 location_1 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/1P0JhfFWZw8= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102019 location_4 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/10e/Xbimpng= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102019 location_2 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/IbsO3i8pDSo= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102019 location_1 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/hVUeBbJOwyY= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102019 location_3 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/AQY8fxNiABI= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102019 location_4 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/cID3BDxKmW8= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102019 location_4 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/b3xcUFkPFGU= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102019 location_5 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/ajhL+h5wAks= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102019 location_2 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/NMqZ1NrX30U= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102019 location_4 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/cxTI8ScW2zQ= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102020 location_4 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/kYJhOeythFw= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102020 location_1 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/lOnH9H9RjB4= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102020 location_3 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/fX24ApmYU34= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102020 location_2 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/sHLGiFIuTQY= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102020 location_1 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/wRTuptRCRS4= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102020 location_4 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/RJZEwW4iDTM= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102020 location_4 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/w+GsMQyDrU4= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102020 location_1 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/jyUW17oebRM= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102020 location_2 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/QQ/JniDBoSM= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102020 location_1 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/DCwa9W8bA0E= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102021 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/r9zslGq3bSE= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102021 location_5 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/7Aw05ort+hU= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102021 location_4 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/Kp7E46gklS0= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102021 location_4 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/+MFRGoulaRk= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102021 location_4 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/IIJm6VoRI0U= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102021 location_2 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/EoPJgwEu+nk= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102021 location_4 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/QkMcJXMmWzI= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102021 location_5 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/V8ZnoROux1w= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102021 location_5 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/idPh/GXSblo= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102021 location_4 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/spbnJG+aFgA= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102022 location_2 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/8yxf6z0EL3o= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102022 location_3 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/oW6Q8PsRCXY= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102022 location_4 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/TjRw1SpIEjM= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102022 location_3 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/J8LPtKYpJQk= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102022 location_1 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/at0vcmvuOSI= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102022 location_2 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/foMO37NT/S8= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102022 location_1 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/iYx2usAfinw= AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102022 location_1 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/f2CohA638Vc= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102022 location_3 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/TgxoxeZq+0g= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102022 location_3 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/Nr0DBJ4BOjo= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102023 location_3 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/VyedrC5XmkE= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102023 location_3 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/oIqF04EnjUc= AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102023 location_2 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/7NWB+iBVx2c= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102023 location_5 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/iczMZBFKW3Y= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102023 location_2 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/xj6KGd4fcns= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102023 location_1 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/VTp1TdRi9iI= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102023 location_4 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/rdTKQWrVPAs= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102023 location_1 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/yPqD1SKbARE= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102023 location_5 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/qNqwAW5y3S0= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102023 location_4 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/0Yo4ZTl/DGs= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102100 location_5 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/7nIlnkGje3g= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102100 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/b/VRSdo/aQM= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102100 location_4 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/tMm7TeW6XQA= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102100 location_1 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/AppWYHaBx2Y= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102100 location_4 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/30cZQDANsBg= AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102100 location_5 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/kX0ref+A0W4= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102100 location_3 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/OoU+B8dWqm0= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102100 location_5 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/WRYriQJyTUM= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102100 location_3 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/n1IcNnu65kY= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102100 location_1 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/lqA3l7+KIRY= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102101 location_4 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/KQOD8Qhk3is= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102101 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/smdh3btULV8= AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102101 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/MIH2jJi76nU= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102101 location_1 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/1PcGG/IkhT8= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102101 location_2 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/aKCbDT3JSFc= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102101 location_1 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/UCfZ21glVn0= AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102101 location_3 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/HC1S48bcwjs= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102101 location_4 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/KBPs2jD48VE= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102101 location_3 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/MwXhswRMUSA= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102101 location_3 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/gnNMgYOL7Rg= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102102 location_5 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/O8dAsCXIKC0= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102102 location_3 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/M0oA16y/gjA= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102102 location_1 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/xQ2waxZVZk4= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102102 location_3 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/vec2eXrLVUw= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102102 location_1 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/BgB1PAyQ+QI= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102102 location_1 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/22skqX11ejg= AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102102 location_5 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/Bbzv/dtX+H4= AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102102 location_5 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/3mYQg9vnanI= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102102 location_4 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/TRBbZOhYx0M= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102102 location_5 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/cADvKAYoPSY= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102103 location_5 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/raZ0GMc6GF4= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102103 location_4 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/NI85+6J9Ujs= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102103 location_4 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/qPEICzNY1HM= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102103 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/IwXov6YK4x0= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102103 location_1 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/FItrMoypwFQ= AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102103 location_1 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/xfdH8ii6VmY= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102103 location_3 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/1IWYzQqNN0o= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102103 location_1 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/YVbQlOnlfz8= AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102103 location_3 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/9hBrz/Ry1Ak= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102103 location_3 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/PbTUcvl3a2E= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102104 location_1 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/u8eEQHjGw2Q= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102104 location_1 product_12 AwEDAAAAAgABAAAAAAAAAP////////9//f/9lNvuKEI= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102104 location_3 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/8ufLAsPSKlU= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102104 location_2 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/speBcvdbWFI= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102104 location_4 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/qzPfq5w38VQ= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102104 location_4 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/MtAOFdDyhUI= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102104 location_2 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/HyBjdsXewSc= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102104 location_1 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/Aqntx/IvjVQ= AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102104 location_4 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/puJZa9vwAxM= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102104 location_5 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/Et0ZgfIdTRU= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102105 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/YT6SikvL4HA= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102105 location_4 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/CBHfVS3OIWE= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102105 location_1 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/TeSJWzWZJUo= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102105 location_1 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/sl9x/1TsDGg= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102105 location_2 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/loPwI1mulCI= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102105 location_5 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/VXC8cWMmvUU= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102105 location_3 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/meG+k+K+2iY= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102105 location_1 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/PSLsEfRKhVQ= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102105 location_1 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/xFkYjLmq4go= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102105 location_2 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/sG28ezvw9B4= AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102106 location_4 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/u7Iew7N6VhE= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102106 location_5 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/jRK/FWgeGGA= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102106 location_3 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/K9VbMl87IhI= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102106 location_5 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/9mVpa8Jp9T0= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102106 location_5 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/+IIQT40pBEU= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102106 location_4 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/QEjV9mwblis= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102106 location_5 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/HAREADcAmjM= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102106 location_1 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/xRNSdYCOjDQ= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102106 location_1 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/2p7Kz8IVyFk= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102106 location_4 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/mQtkX9wT8FM= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102107 location_1 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/Sg996n2ujAU= AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102107 location_1 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/M+cgc82wago= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102107 location_1 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/+ijkb1qVWkU= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102107 location_5 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/DHQX/ZFYbRo= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102107 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/yK5snZdgeE0= AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102107 location_2 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/OW2HBPfeizc= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102107 location_5 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/p7NOKwEsmhQ= AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102107 location_3 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/LaTIRPBJr3Y= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102107 location_4 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/baTMwzeSEwk= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102107 location_3 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/SZnBtl8vJVQ= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102108 location_3 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/Y+Jp8fVoV0Q= AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102108 location_1 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/L4crmkyNYiE= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102108 location_4 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/7zVuLsqdigg= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102108 location_4 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/Sn2YunqkY24= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102108 location_5 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/qz8LM+wpZjc= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102108 location_5 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/lONDKIk5YCI= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102108 location_3 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/HgZu+HJD8QQ= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102108 location_4 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/5Ci4GKPYMgg= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102108 location_3 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/0m+JD4989zU= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102108 location_4 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/Ac+JpJXxmEA= AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102109 location_1 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/5QjAxsfUIV4= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102109 location_4 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/hKd9Mn3AN0g= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102109 location_5 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/jjYyq0zvCA8= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102109 location_5 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/fB2eXq7uImw= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102109 location_4 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/7bSZCgU3O2Q= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102109 location_2 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/1oJ/sAyamB0= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102109 location_2 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/z7HtrjTyc3I= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102109 location_3 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/rh8AHFJHE2Q= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102109 location_2 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/YCxIz1E7c3I= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102109 location_4 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/nFxzT42c5g0= AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102110 location_3 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/gddXIuUKWUs= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102110 location_4 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/h0JZwasZG0o= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102110 location_3 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/RoQCodTAWw8= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102110 location_4 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/HRGR0uUAdiw= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102110 location_4 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/i7hZrUxnjG0= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102110 location_5 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/NxcALOHKVDY= AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102110 location_1 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/4v0T98rF+j8= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102110 location_4 product_13 AwEDAAAAAgABAAAAAAAAAP////////9//NzlARfB8Vc= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102110 location_1 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/f6SEWDwU32A= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102110 location_5 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/Fi/7SYNtOVI= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102111 location_3 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/o+koQ0maC0k= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102111 location_1 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/jLvtij91vAA= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102111 location_2 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/t9VWsHGeQmQ= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102111 location_2 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/FE2ttvzLTDY= AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102111 location_3 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/ridfXKT1/g0= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102111 location_4 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/MtbeKuW3rDw= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102111 location_3 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/2An3LGjVLxw= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102111 location_2 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/xLPzAF186Xg= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102111 location_2 product_2 AwEDAAAAAgABAAAAAAAAAP////////9//ZF5ZgQWQjQ= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102111 location_2 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/XKBIDJH7mUE= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102112 location_1 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/JVQtQk+bKTg= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102112 location_1 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/o+koQ0maC0k= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102112 location_4 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/a450Qz11tnM= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102112 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/PzjlfNzqQC4= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102112 location_2 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/uTsQt/wsSCM= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102112 location_1 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/2rY5qv3rRy0= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102112 location_5 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/CXzSLQRuAn8= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102112 location_3 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/E0U2q03HnD8= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102112 location_2 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/bDQoz6tSNC8= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102112 location_4 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/aDBTShjR10c= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102113 location_5 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/5c4H0CB5tho= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102113 location_1 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/y1MngQbDbAY= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102113 location_3 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/fEnEDvjnGBk= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102113 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/rRLRjFMj8VU= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102113 location_3 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/ZWg/P3QxGyc= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102113 location_4 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/1M6z/ccs02E= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102113 location_3 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/8S36XOx7E0k= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102113 location_3 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/FzBokozQqEM= AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102113 location_3 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/nuhpGgh6zH8= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102113 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/pwL1ch+pT1Q= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102114 location_1 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/tyOi8d3DDRA= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102114 location_3 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/S6MYjB4TJmI= AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102114 location_5 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/OZKiPUpi7FE= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102114 location_2 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/7doyDQ5COBk= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102114 location_3 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/Zloe4v4gRU0= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102114 location_3 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/25B1OfcTYHM= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102114 location_3 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/zJAQLJILLwo= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102114 location_3 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/mnSZ56CBYlo= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102114 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/B0diYOd/ohk= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102114 location_4 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/ZTger/JpxCo= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102115 location_2 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/JFNCc4nEtn8= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102115 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/k5QzzGwR6jk= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102115 location_3 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/SliWyHmcySM= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102115 location_1 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/8RJDrFP5BUk= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102115 location_5 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/l5XndaVgeE0= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102115 location_4 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/sVc3EMYaEzo= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102115 location_2 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/Ea+W8vFnTnA= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102115 location_2 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/N0phsYJ4Q34= AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102115 location_5 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/7v2H8vBWLX8= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102115 location_5 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/pcrlExqMQgM= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102116 location_5 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/ca+7ibSIfFE= AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102116 location_1 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/UlC/V1jQWmw= AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102116 location_2 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/HCJi3Jxwu2M= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102116 location_1 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/mrTkeKSnvgw= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102116 location_3 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/uZ3VrKQVwDw= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102116 location_2 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/LHds0YHyyS4= AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102116 location_2 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/x3XuZ+6TSjo= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102116 location_3 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/6qVpgbm9nEQ= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102116 location_3 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/WPGtOO6GySY= AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102116 location_1 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/jwRbv6kfT1c= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102117 location_3 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/YEaPWhL7Axs= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102117 location_3 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/g/prwWporXU= AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102117 location_2 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/ud1fqbgAqiQ= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102117 location_2 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/EY8wWZQZwW4= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102117 location_5 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/c1kBbAy9exQ= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102117 location_1 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/Xc1bwRX3PyY= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102117 location_1 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/jYzlG9lC+Bk= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102117 location_4 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/WaRJHyU0sVE= AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102117 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/uGAkicHPRx4= AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102117 location_2 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/Hi8+ou2x1F0= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102118 location_5 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/dTXg2720Gwo= AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102118 location_2 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/sZq9XYVwhG8= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102118 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/ant7JLHeyUE= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102118 location_2 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/Ju/5D+TAqgc= AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102118 location_3 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/Lfxtn3rLa0E= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102118 location_3 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/AGdn0OGI22w= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102118 location_3 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/kXm3oSGplyA= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102118 location_5 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/B3/V4iQBKAQ= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102118 location_3 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/NFkqONxcYAY= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102118 location_1 product_14 AwEDAAAAAgABAAAAAAAAAP////////9//6Cgj0OxwTg= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102119 location_4 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/plToNfKBSX0= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102119 location_3 product_8 AwEDAAAAAgABAAAAAAAAAP////////9//HqgUP+5Fmo= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102119 location_3 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/SMPzsyfCinU= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102119 location_5 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/JCDSQg+TV28= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102119 location_5 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/3jnOvTOaHRw= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102119 location_1 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/ybFzUWooGBs= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102119 location_3 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/sLUKGdoFWWc= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102119 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/dNl9P7yGIiA= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102119 location_5 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/aUO+poG4j1c= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102119 location_3 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/QvlGrtjYsgU= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102120 location_5 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/90+glfIqCV0= AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102120 location_3 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/LPidDpsycAQ= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102120 location_5 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/s47m89Q/ihQ= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102120 location_2 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/vLDKX9TgynI= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102120 location_4 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/magcVfa8BXw= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102120 location_4 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/p2FBDGzqaCg= AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102120 location_3 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/luy+clrT1hY= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102120 location_2 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/9eo7MsYQrnE= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102120 location_3 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/JbOLfNPPl1w= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102120 location_2 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/1GvnpUamJ1k= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102121 location_3 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/6eeD/7JAm0k= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102121 location_4 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/8iW7CKnUh2s= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102121 location_3 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/ROXqJgIgTGY= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102121 location_4 product_4 AwEDAAAAAgABAAAAAAAAAP////////9//A90YB2tiX0= AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102121 location_5 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/ibO8gmKwnVo= AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102121 location_4 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/WFZklygF8Ag= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102121 location_1 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/WFMVBTwtUFM= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102121 location_4 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/b7A2Figzw38= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102121 location_4 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/rspUSPwrwxY= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102121 location_2 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/NReLKz/rdDE= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102122 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/Cuix/DlO12I= AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102122 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/fhvjmosVt0A= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102122 location_4 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/7Rc+p2kSW10= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102122 location_1 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/HfS2qeCZl0g= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102122 location_5 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/cyp+QjyYdQc= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102122 location_3 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/PNXxCrkinV0= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102122 location_2 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/JDpXx1+1OUg= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102122 location_2 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/76EoHFeb+Q8= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102122 location_5 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/iJbBGpI4I1s= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102122 location_3 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/a7k2pWfVcG8= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102123 location_5 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/M4h2G2uULzk= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102123 location_1 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/xe+c/57YB0c= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102123 location_3 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/g6wKEzJZxSE= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102123 location_5 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/f7E/Uoc2iFA= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102123 location_4 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/Xc1bwRX3PyY= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102123 location_3 product_10 AwEDAAAAAgABAAAAAAAAAP////////9//TelFFXedRM= AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102123 location_4 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/upvGSurrHjw= AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102123 location_2 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/buwV/ryLzDw= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102123 location_4 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/VO0lgkZFJCA= AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102123 location_4 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/2juIINsV9Es= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102200 location_2 product_12 AwEDAAAAAgABAAAAAAAAAP////////9//3Vy4laswlU= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102200 location_3 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/CzRzjyGOeRE= AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102200 location_5 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/uDt//0xQaGU= AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102200 location_5 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/h63ABryNBwA= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102200 location_2 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/m+N3vXPukik= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102200 location_5 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/VHb5nneCkHs= AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102200 location_1 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/ConRc3/LaF8= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102200 location_1 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/bHS3f7zGfVk= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102200 location_4 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/+p9kbAmvwhs= AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102200 location_4 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/8qq509jULBU= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102201 location_3 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/wD+AFnf45Es= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102201 location_4 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/xkFIGQs+JWQ= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102201 location_4 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/ktdHAKZ55yY= AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102201 location_3 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/NMUzCqLkfjg= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102201 location_2 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/5VmoGbanm1g= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102201 location_5 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/wXorZMD9Dz4= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102201 location_5 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/N09iWDS0yHc= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102201 location_4 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/jLXedmLT90g= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102201 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/mE6zo06AsG4= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102201 location_3 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/573Dk+S8AxY= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102202 location_1 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/jeAakLstq1w= AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102202 location_5 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/uW9fQbTKom4= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102202 location_3 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/iYO4W3zhZAw= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102202 location_2 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/85t+StVIawg= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102202 location_2 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/PtcMltBBoWo= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102202 location_5 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/yz+X5UT4VmE= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102202 location_3 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/9tL4W0NComs= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102202 location_4 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/XD9bK56omE4= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102202 location_3 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/VjXHqAK5K3c= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102202 location_1 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/Mz54gtWHqDs= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102203 location_2 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/35OMNbBZKE8= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102203 location_1 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/3ZpOvYB09UM= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102203 location_2 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/ckRqUtlrX0c= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102203 location_3 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/bQy1rac1Y28= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102203 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/1olJQwYaVTo= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102203 location_5 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/s/l1LCpDr28= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102203 location_5 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/o1yCfMSIh0k= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102203 location_4 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/e/9ULlVvhUs= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102203 location_2 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/6KAxLZKFKgc= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102203 location_4 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/BeI4VpNIrWM= AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102204 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/E0faEuC6ohc= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102204 location_4 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/kZkhd+FgbgQ= AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102204 location_2 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/Rvng8Rip008= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102204 location_2 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/VO0lgkZFJCA= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102204 location_1 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/CuGbgBdb4W8= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102204 location_1 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/m9DPR7iYCjg= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102204 location_1 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/9Vm1GXmW+Fw= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102204 location_5 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/lEJeHcpOXVg= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102204 location_3 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/rgcfw/xM7jQ= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102204 location_5 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/LMyszIy0CTw= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102205 location_3 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/HY0phf3NGRs= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102205 location_3 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/xIbv9HfuORY= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102205 location_4 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/VazWuFUxQAc= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102205 location_4 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/x5XOg3kczkY= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102205 location_4 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/jv/nl1THgSk= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102205 location_2 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/CBXmkG2HNgk= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102205 location_2 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/Sjt26wsdFxs= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102205 location_4 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/gRBBfRdGlFs= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102205 location_4 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/wbLFw9aKk1g= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102205 location_3 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/ttepKTpirGw= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102206 location_2 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/r0vDSL83MBM= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102206 location_5 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/p/nv5UT0hkk= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102206 location_5 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/a8cQHjQn4Uw= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102206 location_1 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/iZLsYKXlWRY= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102206 location_3 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/iYx2usAfinw= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102206 location_2 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/b2ZCkD5vYEI= AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102206 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/5Ziw1150WXg= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102206 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/d56HL9YLqlc= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102206 location_3 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/JdCQlOgnOB8= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102206 location_2 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/4K6K0tY8+SI= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102207 location_1 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/iw9xznQf4wY= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102207 location_2 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/4hwedfQt9zU= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102207 location_4 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/LDoNv5I95y4= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102207 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/7Rc+p2kSW10= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102207 location_1 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/tQoxUa1Q/1c= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102207 location_1 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/17XZka/CIR8= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102207 location_5 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/zEnYzu8SRx4= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102207 location_3 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/VgCdLbu87mQ= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102207 location_4 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/6Hv+grNg3zs= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102207 location_2 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/735qVCCJxyw= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102208 location_3 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/H0ijvPz6+3E= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102208 location_4 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/IaJMtYMjjk8= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102208 location_2 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/baTMwzeSEwk= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102208 location_5 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/787Ufc5ex2Q= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102208 location_3 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/iHKaL4Sju0k= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102208 location_4 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/ExQKL/chimU= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102208 location_1 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/NRI8kaIAtU0= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102208 location_5 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/iwbIOkwROSU= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102208 location_5 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/VKM55Ek7Exs= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102208 location_3 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/PnOKntFwLik= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102209 location_1 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/Sg996n2ujAU= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102209 location_1 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/Y+gHLXSnog0= AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102209 location_3 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/JubYeWrzJyY= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102209 location_3 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/vFXIZ5ke40c= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102209 location_4 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/VUQGmMvvPW0= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102209 location_3 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/xdynHKSsXRY= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102209 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/NQYR3qcovzU= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102209 location_2 product_9 AwEDAAAAAgABAAAAAAAAAP////////9//zIvankEVVU= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102209 location_5 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/Q2KkOar241Q= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102209 location_2 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/f7E/Uoc2iFA= AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102210 location_4 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/X+dVfCX6Vzw= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102210 location_2 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/jOh8YzazNhM= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102210 location_1 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/GeegbsgnuAs= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102210 location_5 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/JMMm5peCPn0= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102210 location_1 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/Kn9qEuNPbVo= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102210 location_5 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/zvW2Nu+P/GM= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102210 location_5 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/MxM4scqVSQo= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102210 location_4 product_5 AwEDAAAAAgABAAAAAAAAAP////////9//CgB3TYfoGk= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102210 location_5 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/GZBnS+LrPGM= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102210 location_4 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/ArMIGEgnilw= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102211 location_5 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/rzuwQN7xVE8= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102211 location_2 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/OOmHQutBMGY= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102211 location_5 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/TI61zESVXF0= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102211 location_2 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/JCbxT6OntmI= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102211 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/YDFKn8ZekVs= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102211 location_5 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/rqNir7nAqW0= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102211 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/atikuFlf5TY= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102211 location_4 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/CAU1vHeQgzY= AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102211 location_5 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/m9FkryhpjAk= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102211 location_3 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/aA4MS1+p4RI= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102212 location_3 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/oGnwt+dIPG8= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102212 location_3 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/HyuJVE4MJ3g= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102212 location_5 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/I7kJ1Z7qfwM= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102212 location_3 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/hffhORdxA18= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102212 location_3 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/E/rMK0OiiEk= AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102212 location_2 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/QcSXX59v5Xs= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102212 location_1 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/fmPNGzDzWiw= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102212 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/U0ktfeh9cjY= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102212 location_5 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/tLVTThjk9FY= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102212 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/j4c8ynl5/y8= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102213 location_1 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/D2naXAn7vwQ= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102213 location_2 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/pKCq6iEhaFw= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102213 location_4 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/XJXzdyS8Xiw= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102213 location_1 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/Nt0NkMl3Zxg= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102213 location_2 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/rpYjpmIn7m0= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102213 location_3 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/akMN3tflpwY= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102213 location_3 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/3mYQg9vnanI= AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102213 location_2 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/gWhLKPVERRM= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102213 location_5 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/PK+GYbOXtgw= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102213 location_5 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/IM/zNxranyM= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102214 location_4 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/3ie4HjDSHis= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102214 location_5 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/i05cUWnL53Q= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102214 location_3 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/xxuNWa7xEVA= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102214 location_5 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/S0tOPWnjUjU= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102214 location_4 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/oi/1fC4SsxA= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102214 location_4 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/d9ACNaRq3FY= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102214 location_5 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/F9+i2d56NRg= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102214 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/azbfNSNQI24= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102214 location_3 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/x875GnUtdho= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102214 location_3 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/PHDRoNQSWEY= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102215 location_4 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/BseSel04Xm8= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102215 location_5 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/GXnHj++I1k0= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102215 location_5 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/62NERFGS4kc= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102215 location_3 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/XXIhNLwBiBc= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102215 location_4 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/7C7TiIks/m8= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102215 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/cTZgUo4SKws= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102215 location_2 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/aDagEhDiTC4= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102215 location_4 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/RsLFHx1NnEU= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102215 location_3 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/OtS11a0R2yM= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102215 location_3 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/GIDdnNgbvVs= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102216 location_4 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/b6b4TwPTMXk= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102216 location_3 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/PMCIqNdEn3M= AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102216 location_1 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/d6TjdLZjFxQ= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102216 location_3 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/zEnYzu8SRx4= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102216 location_4 product_12 AwEDAAAAAgABAAAAAAAAAP////////9//TST8DhtMV0= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102216 location_4 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/izGTGDjoZFU= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102216 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/zfrL9DQM+mc= AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102216 location_4 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/gFX0VaKAeAI= AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102216 location_5 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/S992/oa96S8= AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102216 location_1 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/lqYS9tV4H3U= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102217 location_4 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/M1abvU6O3UI= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102217 location_2 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/TORIE4TcKWU= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102217 location_4 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/cbzvStM6rUA= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102217 location_1 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/a0wRaZUiBm4= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102217 location_5 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/YG7d2biwMB8= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102217 location_5 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/ExQKL/chimU= AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102217 location_3 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/OgMpXLxOeE4= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102217 location_3 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/hfJr3nBPSB8= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102217 location_3 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/FqlB1TKDdmY= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102217 location_3 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/Q5ugxbb1FFc= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102218 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/sDliSlr6y3M= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102218 location_2 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/Lyi2dla3UDU= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102218 location_3 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/73TLLzAS4ys= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102218 location_2 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/5eDShs+2ySw= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102218 location_3 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/H5MoYeZ8iHY= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102218 location_3 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/1IK0VBz652A= AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102218 location_3 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/CQ+WCzvhrHU= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102218 location_1 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/WCLBFKTK5Hs= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102218 location_5 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/rUHMDAk55j4= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102218 location_5 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/9su33S7omUw= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102219 location_1 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/TR1nTn+CKFQ= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102219 location_2 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/DdVwW1TKoS8= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102219 location_2 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/s2/CrrdTgRo= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102219 location_1 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/eTumWV9ONwU= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102219 location_5 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/aJ2FYV8/m3s= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102219 location_3 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/IcnsBv1WBCU= AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102219 location_3 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/Ncypc2UgnCo= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102219 location_3 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/7azpDk/X4kQ= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102219 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/qoKahZQZMng= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102219 location_1 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/b1b3CC5EcwE= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102220 location_2 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/lO3LfV0jOgE= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102220 location_1 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/mQclEF6C7zM= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102220 location_2 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/y+rDKsTpGjA= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102220 location_3 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/h84TvlaxTBo= AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102220 location_4 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/uXj0bxCRVR8= AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102220 location_2 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/TkadcqybtXw= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102220 location_5 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/60YW0tKbryg= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102220 location_3 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/6tQpC5MYenQ= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102220 location_1 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/XFVSgHPazD0= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102220 location_1 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/vSejYJ4W2CA= AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102221 location_1 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/8KNmW+Zq2D4= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102221 location_1 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/L8YY1Um00kM= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102221 location_2 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/fS5XrVqFgTA= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102221 location_5 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/eAn8BJJMeWo= AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102221 location_5 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/26zgscfWDXo= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102221 location_2 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/nT4lq038CxQ= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102221 location_1 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/15ZjMyNO2A0= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102221 location_5 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/uFHSG8MeTk0= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102221 location_5 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/tCv31/N8LUc= AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102221 location_4 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/spbnJG+aFgA= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102222 location_5 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/aFJtKWZ/sAQ= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102222 location_2 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/uf3CbQyOl1M= AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102222 location_3 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/S6MYjB4TJmI= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102222 location_4 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/idGibFqc12U= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102222 location_5 product_9 AwEDAAAAAgABAAAAAAAAAP////////9//TOkcujtDhc= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102222 location_1 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/NxBhJ1Wl6nY= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102222 location_4 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/FBIPuZzSeCQ= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102222 location_4 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/NyK9FE6hGhg= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102222 location_3 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/PZHiLpbciBg= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102222 location_2 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/skiNbUCPiCw= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102223 location_3 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/KT4w2+oiKmU= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102223 location_1 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/zea2eYmb3U8= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102223 location_3 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/gBbNQWl14yE= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102223 location_2 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/iN36dGCtdW8= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102223 location_3 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/vfYn3/immiA= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102223 location_2 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/2yZvQZ1FJgs= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102223 location_3 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/zpnmp63grhw= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102223 location_3 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/10e/Xbimpng= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102223 location_2 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/5I+SPG9jyVc= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102223 location_2 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/t3XMIOdkd2w= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102300 location_5 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/5Lb2EDZZSko= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102300 location_4 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/EuA7iasMjBc= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102300 location_2 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/jf72odbmwV8= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102300 location_1 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/9Vm1GXmW+Fw= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102300 location_2 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/wb7IRcKJPls= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102300 location_4 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/3pPmHUWuLXs= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102300 location_1 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/a7p7cZxrClI= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102300 location_2 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/Y9W3RiXtWBE= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102300 location_3 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/JLJWxUAjM3M= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102300 location_1 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/dKi+4wmQKQg= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102301 location_2 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/ovPw7CRUYHw= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102301 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/+0s00yblOl4= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102301 location_4 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/rgmZ9E3GGBE= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102301 location_1 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/eVukUy1In2o= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102301 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/W2ohDhfTkwQ= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102301 location_2 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/MIH2jJi76nU= AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102301 location_1 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/AuJBnIcWNhY= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102301 location_5 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/Im2APXA61AY= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102301 location_4 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/jmChtm+zA1U= AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102301 location_5 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/xpSyiqYESBA= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102302 location_1 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/ldio7MH75VI= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102302 location_5 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/S5Y7xafHGlM= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102302 location_1 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/DHup2QUlbxk= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102302 location_2 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/HbCIy+WB8V4= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102302 location_1 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/vpxBKUshIXE= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102302 location_2 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/R/R87JPsbzg= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102302 location_3 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/A8z2GOGAfHU= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102302 location_1 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/ausn4yp9xlM= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102302 location_1 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/NnO/8NdTg1E= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102302 location_1 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/wZixDuyHbmg= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102303 location_5 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/uUxFk0n/wVc= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102303 location_2 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/60YW0tKbryg= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102303 location_5 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/WcCAA02VfyM= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102303 location_5 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/ZQlq3BkkVkU= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102303 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/j4r4v4e0uWA= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102303 location_4 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/vu1NAfekLnE= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102303 location_1 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/IG+T84wVnTg= AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102303 location_2 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/ChdLbmvCPQ8= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102303 location_5 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/ujpDS0MeoR4= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102303 location_2 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/k5qFeIJxqDU= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102304 location_1 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/XR64le8uszc= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102304 location_5 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/kBI/r2YQ5TI= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102304 location_5 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/8u6761dMd0o= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102304 location_4 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/xVnIvcM63Uk= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102304 location_3 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/wKk8PlXz2To= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102304 location_4 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/20REVvQQgBU= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102304 location_5 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/FdP0YhuHnD0= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102304 location_5 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/HHxVo7li7l4= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102304 location_3 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/kpI0s4kPNlI= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102304 location_3 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/UD6i6gf/iAI= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102305 location_5 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/L8sP46WoeTo= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102305 location_4 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/Mylw39K6PmQ= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102305 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/cYXmF5b6jCI= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102305 location_4 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/pWLFNs0sGGw= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102305 location_3 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/8lrKkjilpQg= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102305 location_2 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/difz4OuSmmw= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102305 location_3 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/mHXglwjp9nM= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102305 location_3 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/J9HB/EnJ1E0= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102305 location_2 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/nmYSN57Nei8= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102305 location_5 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/lE97+rNNcVo= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102306 location_4 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/vUNYYhnz3VM= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102306 location_5 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/wQCbYffDURA= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102306 location_1 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/o7i8nGUz/Ww= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102306 location_2 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/p3s3O8pQTxk= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102306 location_3 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/E0MJDxed5wU= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102306 location_3 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/NvzoroIJlho= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102306 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/HzXacbeyQUM= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102306 location_4 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/e4Km+/4h3U4= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102306 location_4 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/6YVGT/rGrk8= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102306 location_2 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/6xnHnIgTX1s= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102307 location_4 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/6FEdK3bmBBI= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102307 location_2 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/GLBx/kxq23g= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102307 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/E5N8lI1hW14= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102307 location_5 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/CBXmkG2HNgk= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102307 location_2 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/VazWuFUxQAc= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102307 location_1 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/lNENWUoyYQ0= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102307 location_1 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/wuzl0/Veiwg= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102307 location_4 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/XbhS2KdIXDA= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102307 location_5 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/EZ0PBS9Ur20= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102307 location_1 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/jT+YRY21CGA= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102308 location_4 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/GXyI320myxY= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102308 location_2 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/i+mJ5xhignI= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102308 location_2 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/uKanK4YIgVk= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102308 location_5 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/9OuU2gwqh3w= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102308 location_1 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/jusfZfMvvDQ= AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102308 location_5 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/3bSbTcBCrV4= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102308 location_5 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/OHfIVh4zuzA= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102308 location_4 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/pzAmGBpiqU8= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102308 location_5 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/yRBaIN4t2Bw= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102308 location_5 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/bqtOOY1nl0c= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102309 location_3 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/wRlTrvGOmnk= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102309 location_1 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/DZQWfIbeD2Q= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102309 location_3 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/9FZyBS45JUY= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102309 location_3 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/PMPYzVUgPio= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102309 location_2 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/e0AhmWWYdWw= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102309 location_1 product_1 AwEDAAAAAgABAAAAAAAAAP////////9//qPj0oRTEGg= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102309 location_2 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/RUcSrkGPijI= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102309 location_4 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/OForEbxcjSk= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102309 location_5 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/s5BDe5qH5Gg= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102309 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/aUGAU/EYths= AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102310 location_3 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/gb4U2z/EuTA= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102310 location_3 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/fjtrocsDxwI= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102310 location_5 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/kSHNde6aZiI= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102310 location_4 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/U8SxzTcJYyg= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102310 location_1 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/+WXq5LZBrB8= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102310 location_5 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/oO0wJPenb2I= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102310 location_4 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/SUFMO3CWZFQ= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102310 location_1 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/U+8z0Uz9Ejg= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102310 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/K3anmAfw+nE= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102310 location_1 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/tjstetpt50Q= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102311 location_3 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/hFFQ8lf5DC4= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102311 location_5 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/THR2uBr7z3Q= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102311 location_3 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/95ajNy9dgz4= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102311 location_4 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/r75EoMqZvRg= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102311 location_1 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/BOrbVSPLpV8= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102311 location_2 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/uUCbqGlTpks= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102311 location_2 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/FTcY9VdK3xo= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102311 location_5 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/9frpsYB6Bhg= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102311 location_5 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/zuPOX/zPJy0= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102311 location_1 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/S6Y3oTecsX0= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102312 location_4 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/YG7d2biwMB8= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102312 location_5 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/bnafkZ2ioEM= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102312 location_2 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/c3F10XIpbAg= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102312 location_2 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/nwUmQPpdVi0= AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102312 location_3 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/QbtXn1yT2Tg= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102312 location_2 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/eOy2VgwLPSQ= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102312 location_4 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/hq+WYsTZCDE= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102312 location_2 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/fWEo9uNkzW0= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102312 location_4 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/OMO3e6rdv2c= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102312 location_2 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/q1kcj+BcKWE= AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102313 location_5 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/lnzcMfhGlDQ= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102313 location_2 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/ajuV0hZ69TU= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102313 location_2 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/A5fJjT656Ag= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102313 location_4 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/94l7A5Z10wg= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102313 location_3 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/9kA3mAtwejE= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102313 location_1 product_1 AwEDAAAAAgABAAAAAAAAAP////////9//L5vCqDS0Rg= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102313 location_4 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/HZhEN96Eexg= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102313 location_2 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/lq+Wy6ZWZj0= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102313 location_2 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/P77IL6B9wCE= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102313 location_5 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/HamgqeMPxxU= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102314 location_2 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/rsUckMFDJXI= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102314 location_5 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/S5z1m0RlNS4= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102314 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/7WJuz+bKDhc= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102314 location_3 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/scbhF1AsI3c= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102314 location_3 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/KAkhBuZHVjA= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102314 location_3 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/VgCdLbu87mQ= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102314 location_4 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/5KOnRDI9Al8= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102314 location_1 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/SvMGix/Ya34= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102314 location_2 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/WibKhbQvvCM= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102314 location_4 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/dJn9jHUOpTg= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102315 location_2 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/+Ai2i0nUrXo= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102315 location_2 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/hud9+ZPQ/Vk= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102315 location_4 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/mOLDv4NA/xs= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102315 location_2 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/qnZSpcT7Z2I= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102315 location_4 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/iwHmvgKHSgA= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102315 location_5 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/OAmhe6WEn0A= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102315 location_4 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/F7y7u0nO4ic= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102315 location_4 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/5J1jLgI0fic= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102315 location_2 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/L4a1aYGMllo= AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102315 location_4 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/tj48ONmHdlQ= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102316 location_3 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/dYmMKSs/ZXA= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102316 location_4 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/YmEDHyfmzh8= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102316 location_2 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/FRBZI/epwko= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102316 location_2 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/6I2w1v/YG1E= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102316 location_2 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/c51tbWM8aWg= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102316 location_4 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/3FMQkqQ4RmE= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102316 location_4 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/WUrMQYweFgA= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102316 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/qXyohSwaNw8= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102316 location_2 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/6vKyF/YUzHg= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102316 location_2 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/um7+4qpgkjE= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102317 location_2 product_7 AwEDAAAAAgABAAAAAAAAAP////////9//6kfuMeILUc= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102317 location_3 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/G1OXBFyhWzo= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102317 location_2 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/JRCk6ZZQGBg= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102317 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/7iSwHLHHjSE= AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102317 location_3 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/osAY/2SMTRI= AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102317 location_5 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/wRlTrvGOmnk= AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102317 location_5 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/s/BWbzMUIA4= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102317 location_5 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/BhttbWkBmws= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102317 location_2 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/t6VNHM7MrFw= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102317 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/f+vt/sYNWW0= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102318 location_5 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/TBv36L6iolI= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102318 location_4 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/gce6X5dBb2c= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102318 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9//zfj0in8z1c= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102318 location_5 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/27lOGYAKjWE= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102318 location_3 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/XH8eEaHS4iY= AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102318 location_1 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/E8nFkXQGpRk= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102318 location_4 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/tXoZ6+Hh7S4= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102318 location_2 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/hKsSZECOqUw= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102318 location_5 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/NF43guoaMBk= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102318 location_2 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/knmCaftT4X8= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102319 location_3 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/rpe4RGwesTE= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102319 location_5 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/TO/dFCMjKzI= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102319 location_1 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/8bfCEfaXkUw= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102319 location_1 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/cdtu/U62bHY= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102319 location_5 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/VlV39FGfBA4= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102319 location_3 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/McgZxI+AVBE= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102319 location_2 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/9zjnQv4nCVU= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102319 location_2 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/6tNiVVfMWTE= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102319 location_3 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/xB+eHto6oCM= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102319 location_1 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/K5g4ybfm2zc= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102320 location_3 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/Lbo+XmdHPz0= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102320 location_2 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/Wiwu6gukLx4= AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102320 location_3 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/kl8Jk+MgFSc= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102320 location_5 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/Nn7RaNBjb2s= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102320 location_5 product_13 AwEDAAAAAgABAAAAAAAAAP////////9//bu/wCPSITg= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102320 location_3 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/OfLdCN5mgDE= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102320 location_5 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/sg3haRfrYgc= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102320 location_4 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/qZijnq3tBQk= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102320 location_5 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/QCMHEdBmDDM= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102320 location_5 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/uQmHqp8Scno= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102321 location_5 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/G0XUS+I0HGM= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102321 location_3 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/poafNgduXWw= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102321 location_4 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/0wBOjyOVsWM= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102321 location_5 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/Y4tLq1Y25Fw= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102321 location_2 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/cn+A3wnwTjE= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102321 location_4 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/Q9ls32MxTWY= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102321 location_1 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/DPJvh0LfxmI= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102321 location_1 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/HiLBY/ShRX8= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102321 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/o5g01FrRD10= AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102321 location_1 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/1DuoHG/exgY= AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102322 location_3 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/bbf5AwpLuVI= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102322 location_5 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/Pq7WsHomlhk= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102322 location_3 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/01ANdzdR3nY= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102322 location_5 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/qAAhE7W71CU= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102322 location_4 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/OuA/ndIXzD8= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102322 location_4 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/pB4AH8TCtEc= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102322 location_2 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/8WEzjtEphXg= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102322 location_4 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/gI7Gz1Z8ByM= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102322 location_5 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/SRt2ShngoTo= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102322 location_2 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/1/+Xb0JTY1s= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102323 location_2 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/bL3//ToZrT8= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102323 location_5 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/D6sbR/X+j2Q= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102323 location_1 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/PprvQhYISi8= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102323 location_3 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/j8190fZefHM= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102323 location_5 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/LzgfBaC47V0= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102323 location_2 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/6yZKQ+Mi9Go= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102323 location_5 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/T4PnFB/KjVw= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102323 location_2 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/hNH87NR9Jiw= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102323 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/GUnY0tMX/3s= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102323 location_1 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/02DAx9vfIhU= AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102400 location_5 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/573Dk+S8AxY= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102400 location_3 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/DoGgxwfG70A= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102400 location_4 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/d1CMgaiZBQc= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102400 location_5 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/t3MtJ3Wo4FE= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102400 location_1 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/MoEUQALMU3E= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102400 location_1 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/xUtgGZ22y14= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102400 location_2 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/jTrAwbx6hQU= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102400 location_1 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/4gej9ywykRY= AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102400 location_4 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/MMFF4+LQzEg= AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102400 location_2 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/9/b3R0vNoXg= AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102401 location_5 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/obL4EbgqzhE= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102401 location_1 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/Dd9I5uKwRRM= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102401 location_3 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/8RYkM2mR0zU= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102401 location_2 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/icfti7OZBTM= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102401 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/JfC1TrUG3Es= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102401 location_1 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/LsbRPuxoYgA= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102401 location_2 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/sUBqVMuZfDI= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102401 location_3 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/NTDy9LwmiAg= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102401 location_4 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/BwJY4NMRl14= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102401 location_2 product_1 AwEDAAAAAgABAAAAAAAAAP////////9//ZTuj7ILMQ8= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102402 location_5 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/sKVjRuEx+U0= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102402 location_4 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/2oEBjEenaiQ= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102402 location_3 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/DkHw8dD/PwE= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102402 location_4 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/iDtuiEMGzDg= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102402 location_1 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/yZUHISVxMy8= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102402 location_4 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/Qd2Uiq2D/gY= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102402 location_5 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/YGSm/7Un1H0= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102402 location_1 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/5Gv2TxwJUT8= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102402 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/qX4PmI0CaAs= AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102402 location_2 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/Ktplzo/RC2w= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102403 location_3 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/1XTXYa6kZyU= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102403 location_3 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/qeuf/C86RRI= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102403 location_1 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/rXk54CNFpUo= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102403 location_4 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/4HVu2wjSTlU= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102403 location_1 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/ln2gUPb6IXk= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102403 location_2 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/95liMtBPMFw= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102403 location_1 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/MR6e9sHe7TY= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102403 location_4 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/MXTZEyYwEAE= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102403 location_3 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/e5qA4zsJgCE= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102403 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/ed0oSr1+jmo= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102404 location_1 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/g6GGioErymQ= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102404 location_4 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/QNWkjuzRrXE= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102404 location_3 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/Y2lZhpTodF0= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102404 location_2 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/LEwK4AF/RAg= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102404 location_4 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/Ed2DxmBqJn0= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102404 location_5 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/bkmgZGowZEU= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102404 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/bhqUgHaXlk0= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102404 location_1 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/Yh3a7xjRfUw= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102404 location_2 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/UzZK8s/wgA4= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102404 location_2 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/OuA/ndIXzD8= AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102405 location_1 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/M6YsbveQ/QQ= AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102405 location_3 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/XEEANax4KDc= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102405 location_4 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/pOrx6rHjfG8= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102405 location_3 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/cDQ/+FyTA3Q= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102405 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/Fx47GsWRWBM= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102405 location_4 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/3eDpvMiYZBI= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102405 location_3 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/FOm5OFZ59Vo= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102405 location_1 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/ECqGJTdZ/lw= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102405 location_1 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/Xq64t1SgpT4= AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102405 location_2 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/KKoPuoemtXY= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102406 location_4 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/Qe1AVc77oU4= AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102406 location_3 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/I1hC95UvpHM= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102406 location_2 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/4lUbIoYQq2M= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102406 location_5 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/e+O6W2MkjWA= AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102406 location_1 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/yfHysgJomWU= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102406 location_4 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/U+T8T9FL5jU= AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102406 location_2 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/J5A4ChepR0I= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102406 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/ziQ/eLw4sjg= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102406 location_1 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/ALwro6mkEy8= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102406 location_5 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/bLZXx5lhXQo= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102407 location_4 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/2bic3LP36Ac= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102407 location_4 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/RL/MMZkXnzI= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102407 location_5 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/wZixDuyHbmg= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102407 location_4 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/25J/ldIvKW8= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102407 location_1 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/WuAIkpdK/wg= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102407 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/FoiGza8Hjlw= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102407 location_5 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/nZQDYsj6aiU= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102407 location_2 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/NFE1K/06/iM= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102407 location_5 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/kLTkrY7EwAI= AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102407 location_2 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/+df+AHL8jR8= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102408 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/KBPs2jD48VE= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102408 location_5 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/RvxDrAnFwmo= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102408 location_4 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/9GmjRz0n+y0= AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102408 location_5 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/KrrBY4pfDXc= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102408 location_5 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/klVUnaiaA3U= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102408 location_5 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/90uf4CEsghs= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102408 location_3 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/JRCk6ZZQGBg= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102408 location_3 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/qYItY0tu0Fg= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102408 location_1 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/s2/CrrdTgRo= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102408 location_5 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/MIZ/pAFSkyM= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102409 location_2 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/Ii0aNz0y5HU= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102409 location_3 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/yzb/2VMYMTc= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102409 location_2 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/1brBfRKUMh4= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102409 location_2 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/dFlq2YKfdE0= AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102409 location_4 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/xmgv5hrWQQk= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102409 location_1 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/uRdCGQwF1Hg= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102409 location_1 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/vSaDIrIAG1Q= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102409 location_2 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/vrwlSupsiDY= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102409 location_4 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/EIuBM2+DEzE= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102409 location_1 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/FDAGaDMpfFQ= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102410 location_5 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/eb9Kfy8/7lE= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102410 location_4 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/nxu1vk5uYCE= AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102410 location_3 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/1kTzXe3LmVI= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102410 location_2 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/xqCXHnvHUlo= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102410 location_5 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/iUB4Rn/cgF4= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102410 location_3 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/M33QVSthZR8= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102410 location_2 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/35UFDx7/oj8= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102410 location_2 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/rMLq3zytrgE= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102410 location_4 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/TfY5Ov/BfBc= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102410 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/SksUH8hlx3E= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102411 location_5 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/OInP/rTAQiQ= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102411 location_5 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/kG1tcaZJG2k= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102411 location_2 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/kFZ0mr9sFBk= AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102411 location_3 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/aUSfPT3WCWs= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102411 location_3 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/ngjmCTIlS0U= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102411 location_3 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/gBbNQWl14yE= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102411 location_4 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/HjryXeutVjY= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102411 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/15Le+NGSslg= AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102411 location_3 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/9/b3R0vNoXg= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102411 location_1 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/gb4U2z/EuTA= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102412 location_2 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/aoCIcSFeORg= AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102412 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/NZQQPNbPQ3Q= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102412 location_3 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/R74pFhujI1g= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102412 location_2 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/OZRR1uJIEEY= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102412 location_4 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/pvA2PWByCjM= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102412 location_5 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/CwjYVO0CTy4= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102412 location_5 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/AaCRDNH0QxA= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102412 location_4 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/h1SdgwG0ong= AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102412 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/xUtgGZ22y14= AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102412 location_1 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/0w1z2ubZ/GA= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102413 location_4 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/Z7Bde5OIoHs= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102413 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/5PH9/yT0g00= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102413 location_1 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/r8f2Qsno0gI= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102413 location_5 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/xGMKqAl0hW8= AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102413 location_2 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/NnO/8NdTg1E= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102413 location_3 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/j2sEBoTo9H0= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102413 location_1 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/lzeXxsokUHE= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102413 location_4 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/qXSqIkaJ6y0= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102413 location_3 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/uhXig2l4a3Q= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102413 location_4 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/D1+E2tb9Uj4= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102414 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/XKBIDJH7mUE= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102414 location_2 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/MLNb1dLwYQQ= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102414 location_1 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/DLrCzClwvG8= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102414 location_3 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/Vfv4RYz5MHE= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102414 location_4 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/UsCJ1lUsXks= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102414 location_2 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/YFfR73LpR0Y= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102414 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/p7NOKwEsmhQ= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102414 location_2 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/2eziLrfpQ2k= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102414 location_1 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/7Ap4+pDY8Ss= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102414 location_3 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/a6asCY4lxCw= AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102415 location_5 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/qskSK5ugEzE= AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102415 location_2 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/ujpDS0MeoR4= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102415 location_3 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/5gz+iy5jNV4= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102415 location_5 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/8977Rawotwc= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102415 location_4 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/+Xpdlbs3EFI= AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102415 location_3 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/Bi2qLQuSjBs= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102415 location_4 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/kgHpM0ie9RY= AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102415 location_3 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/jMROFjhVckE= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102415 location_2 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/CjORYbO9rl8= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102415 location_1 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/3jnOvTOaHRw= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102416 location_1 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/aOTf/+9bcE0= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102416 location_1 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/6I9SLqmw6VE= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102416 location_4 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/WZaD574fnx4= AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102416 location_5 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/kN0j5LK8tAo= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102416 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/85YFeu/l9Ds= AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102416 location_5 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/XE5MOcpxmzk= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102416 location_5 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/otz2rONLUiU= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102416 location_1 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/zqxwohMwYQo= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102416 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/Cft9WcDR9z0= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102416 location_1 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/gjFlfuaIIhQ= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102417 location_4 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/vCynq8GTcU4= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102417 location_5 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/WLSXAPgs4Hg= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102417 location_1 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/3FisV+hJ3yw= AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102417 location_3 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/5MZmFVt+wBQ= AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102417 location_2 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/opyQ9jPRRzE= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102417 location_3 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/EBv1Q3iQ7hY= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102417 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/AbEmi9bYSgk= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102417 location_2 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/NKBiB04Aywg= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102417 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/EA03xnLO2y8= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102417 location_2 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/zLoopbJ2tB4= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102418 location_3 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/OAmhe6WEn0A= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102418 location_3 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/RTPdacS4/10= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102418 location_3 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/UyU/wNEk+yM= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102418 location_3 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/cvALpzOgIR8= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102418 location_3 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/wmZFI3+KyHM= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102418 location_4 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/vyGsOqWM7wo= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102418 location_2 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/FY8C8AEcCWM= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102418 location_1 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/QI0Uo9npvUI= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102418 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/23N5DADyclU= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102418 location_5 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/a0lNxRGeGyk= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102419 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/jQRAE6BroTQ= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102419 location_3 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/zjH301hifGA= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102419 location_3 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/9+XOGyAl7ko= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102419 location_2 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/k1CN9BQy3Xw= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102419 location_4 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/YGSm/7Un1H0= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102419 location_3 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/iGifNhzbJCo= AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102419 location_5 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/W+fi/QbYgCU= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102419 location_1 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/OyFbNFS8hVc= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102419 location_4 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/W43/BXyxhQE= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102419 location_2 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/2YulRUQXdUs= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102420 location_4 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/pA1KJXq2zTw= AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102420 location_4 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/AG3UWluLMQ4= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102420 location_1 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/05vwCCW/wS0= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102420 location_5 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/JmDglvDA0RA= AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102420 location_2 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/doRdr3CBth0= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102420 location_5 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/GyKvctZeR2I= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102420 location_2 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/c6zpY5MdRTk= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102420 location_4 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/YiN4Kk1OZ3U= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102420 location_3 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/5ldsqOrP6n8= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102420 location_2 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/88LIRU7MXns= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102421 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/PzzwhSerYUA= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102421 location_4 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/AppWYHaBx2Y= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102421 location_3 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/XcToaiyzWys= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102421 location_5 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/RQkYtvHW4Dw= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102421 location_2 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/573Dk+S8AxY= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102421 location_1 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/9frpsYB6Bhg= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102421 location_3 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/G0RwFUktmW0= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102421 location_1 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/3PBw+sL7fQY= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102421 location_2 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/OMO3e6rdv2c= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102421 location_3 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/7izvWdkAAS4= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102422 location_4 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/Idtw0SuA5HI= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102422 location_3 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/Q7u51zRC+QM= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102422 location_2 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/00dYSte8tkE= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102422 location_1 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/NSsffQOaTyM= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102422 location_2 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/dd9kFtm0eQM= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102422 location_5 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/YK6VIkLJyF0= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102422 location_1 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/nXhHrNpCTWQ= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102422 location_4 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/RRzfCKL4hXQ= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102422 location_4 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/fWxy+/E1jT0= AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102422 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/MPv7g6+x034= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102423 location_1 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/AaCRDNH0QxA= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102423 location_1 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/xi8pqIMAXXk= AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102423 location_4 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/Q20N+J6teRI= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102423 location_1 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/8k19pznZ/wo= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102423 location_4 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/Wi2JM33bDhQ= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102423 location_5 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/GokS9STmmQ0= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102423 location_3 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/IrLhouDCeUs= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102423 location_4 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/4FzKH1QopFc= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102423 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/eRHUCoYK01Y= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102423 location_1 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/+c3V0e8u7VY= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102500 location_4 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/jcrdezOa8FA= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102500 location_2 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/WRYriQJyTUM= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102500 location_2 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/11CCEMieHTg= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102500 location_1 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/n5oatIftLh8= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102500 location_3 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/5PIj9mXy/UY= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102500 location_2 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/nDmHzpBypEw= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102500 location_4 product_2 AwEDAAAAAgABAAAAAAAAAP////////9//afAa6n8QR0= AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102500 location_2 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/T49V8L76WGc= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102500 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/TIy6r9s4uRg= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102500 location_4 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/p//SMlvyayM= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102501 location_5 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/9qA63aL5/wI= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102501 location_5 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/bU6EPMtqWCk= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102501 location_2 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/iNBCSjpRRFA= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102501 location_2 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/FRBZI/epwko= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102501 location_3 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/L+r3CDfS5Dc= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102501 location_3 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/BaroYWrrg1A= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102501 location_3 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/1uky/6Buej0= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102501 location_4 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/CLMrAoEEIAk= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102501 location_1 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/BgQHfbRtVC0= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102501 location_2 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/FtrokeiuyCo= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102502 location_1 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/4+SSOAbEkEU= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102502 location_3 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/S0J0MaKb9Hc= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102502 location_2 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/539Uv6NtrBc= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102502 location_3 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/DzqxhiNqjA8= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102502 location_3 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/5gz+iy5jNV4= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102502 location_4 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/O2uUX/QV5U4= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102502 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/SlWkZzoFigc= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102502 location_3 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/zooWa0z4wgE= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102502 location_1 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/RE4Iy8Mprkw= AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102502 location_2 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/ggzqcBRJXxI= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102503 location_1 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/bQs3vCjl4Fo= AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102503 location_4 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/8Hk5YaDs/hE= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102503 location_2 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/ZCEspYeZkgU= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102503 location_1 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/3NdkX6DO/0g= AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102503 location_4 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/3NoeSBUc1B4= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102503 location_3 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/Enw3YTKC9ks= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102503 location_1 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/TgxoxeZq+0g= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102503 location_2 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/8KNmW+Zq2D4= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102503 location_2 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/waY7VfYGfSI= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102503 location_4 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/OApQkXsryBk= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102504 location_1 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/hEs0QW6T2Hc= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102504 location_3 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/iOpWhf+JfRA= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102504 location_5 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/nIixmg026H8= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102504 location_2 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/45MMwms8Z3M= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102504 location_5 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/FPKUS7Olsh0= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102504 location_4 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/VujnxauWJkw= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102504 location_5 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/P9Fsr7mXNF0= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102504 location_4 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/Aq3hNd8t9mA= AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102504 location_5 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/rHPMN43y+Ck= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102504 location_3 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/GOl37kPkAHg= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102505 location_2 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/TkMkrmRUwGo= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102505 location_1 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/u3Z7BfJKBD8= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102505 location_2 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/1o+X/xYO9x8= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102505 location_5 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/7Aw05ort+hU= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102505 location_3 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/lCADQOY0IlI= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102505 location_2 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/nXr90LwSdSM= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102505 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/HVLJSG5E6WE= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102505 location_2 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/CJf+Ub3BsAs= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102505 location_4 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/1r+do89tUEk= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102505 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/HXGikpsHgjA= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102506 location_5 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/l1n54GaaY3U= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102506 location_2 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/zx0+4C3nrFU= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102506 location_4 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/5w+eNqEM6Ak= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102506 location_2 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/o5sj2rVMa1c= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102506 location_4 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/PW350nGc1zU= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102506 location_4 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/Bi10H4biBys= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102506 location_4 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/O4OS07Oe/WY= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102506 location_5 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/+EzCrVaPYBs= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102506 location_2 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/g+VvSAXJ9g4= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102506 location_2 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/MMQ+fxkbmQo= AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102507 location_1 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/pC077IV9/Tk= AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102507 location_1 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/4b+zfkdQEB4= AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102507 location_2 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/Byhhh3BDHEo= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102507 location_1 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/Sx/Ii9wOnns= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102507 location_5 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/9goyxo8zils= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102507 location_5 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/epbB+Wu9hQM= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102507 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/z0/wEAk4fAY= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102507 location_4 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/m79f/01sWjE= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102507 location_1 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/zT8O+rLOzR4= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102507 location_5 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/12X+jsG99lk= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102508 location_1 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/aZIHF3wiq3U= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102508 location_5 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/BELXZ2KdXXs= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102508 location_2 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/GFqS/Gdq3WA= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102508 location_2 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/P5YGiLNPLXs= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102508 location_1 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/Op3Hst3w8Ss= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102508 location_4 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/aL2FBvXjzDA= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102508 location_3 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/fLgnz4y81lE= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102508 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/5qWdSQrhHV0= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102508 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/i4TJQOEIEzI= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102508 location_4 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/NGt/07Ml2lM= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102509 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/kPxwy2jRXxg= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102509 location_2 product_5 AwEDAAAAAgABAAAAAAAAAP////////9//G3TyX4EHyU= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102509 location_5 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/7C7TiIks/m8= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102509 location_2 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/tWFZ6Lp1ZwI= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102509 location_1 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/aFEqwUztvXM= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102509 location_1 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/ydW/AH4Pu1U= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102509 location_1 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/S/c89sUKpBY= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102509 location_3 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/jLPR8TQxtXY= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102509 location_5 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/rUHMDAk55j4= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102509 location_3 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/PaMsfATLoRg= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102510 location_2 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/5T0w3QQGGho= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102510 location_1 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/em0CWkrGlgk= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102510 location_5 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/r8aCDRnAKAg= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102510 location_4 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/y4ezL/k/GA0= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102510 location_3 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/JAr5rsoe42M= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102510 location_3 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/oDvDE441mxs= AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102510 location_3 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/AIM4dqPtAFA= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102510 location_1 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/6IH98Iz5wnY= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102510 location_5 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/LzE5Ob+YH2c= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102510 location_1 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/9meDnZIMQSk= AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102511 location_4 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/8bfCEfaXkUw= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102511 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/YOmU4NyzvFk= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102511 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/0ZldrjRXU0g= AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102511 location_1 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/2pcBOIoMk0M= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102511 location_2 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/LBza5xZiwSE= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102511 location_2 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/n+KWq9cS7U4= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102511 location_2 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/VVO5++dfWiA= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102511 location_1 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/E4i5G1TNkys= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102511 location_4 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/XtEpJveM42M= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102511 location_2 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/vwb4La19zmc= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102512 location_2 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/lRUQIvzNmkQ= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102512 location_5 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/FJebRhL7dXE= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102512 location_3 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/gBbNQWl14yE= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102512 location_3 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/EcxNT/FwJGY= AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102512 location_1 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/wu0FvZ3cWHg= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102512 location_3 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/49Wq8LpA9lQ= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102512 location_1 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/3UNXSKWV6Ro= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102512 location_4 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/c+v3UTTXLno= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102512 location_1 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/Kl15sBKGvwg= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102512 location_5 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/oWN/0z/AfW0= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102513 location_4 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/W04dhrurcQg= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102513 location_5 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/AHcKbemN+m4= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102513 location_2 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/Me/kAvkSyyg= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102513 location_2 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/6A32Pv9wM0M= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102513 location_4 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/+wLDXTg5plI= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102513 location_1 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/iRKuQiXO9EI= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102513 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/jqJEiA7nHnM= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102513 location_1 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/mBNbM9ygr2g= AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102513 location_3 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/aqdOQ2L3qBA= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102513 location_3 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/Bg0To9He+l8= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102514 location_1 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/PX270LM4nH4= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102514 location_3 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/NsoXEoSlqCM= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102514 location_3 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/AhDV7AMXCEA= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102514 location_1 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/ngXuPN7Ponk= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102514 location_4 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/BduevXWsu3A= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102514 location_4 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/uObiG+NgG3A= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102514 location_3 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/O+vMqEFUNiw= AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102514 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/aW520WVj1wE= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102514 location_3 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/k7eZIquQnBo= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102514 location_3 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/w8T6sByD0CY= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102515 location_5 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/jqJEiA7nHnM= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102515 location_1 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/Ehpti9DFKTE= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102515 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/tasbbzssrgI= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102515 location_4 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/kfSZqe0hsAQ= AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102515 location_2 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/iz7xpV6x4RU= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102515 location_4 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/FTcY9VdK3xo= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102515 location_5 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/M+m9sVnmujo= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102515 location_4 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/CxcU7rvEa3I= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102515 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/wOx0RXpqQ0M= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102515 location_3 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/1MRugQKdBRE= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102516 location_5 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/BgG8Zysj3To= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102516 location_2 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/J5A4ChepR0I= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102516 location_2 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/fo0ZOfooiC4= AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102516 location_3 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/m/7ZgBjaf1A= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102516 location_1 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/9l1ineQypR0= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102516 location_4 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/4e5NQlaheAQ= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102516 location_2 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/2ahDA/VraHs= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102516 location_3 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/bSRGFZiiMyQ= AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102516 location_2 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/MPX2xd4uFmY= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102516 location_3 product_13 AwEDAAAAAgABAAAAAAAAAP////////9//rsZnB52z2o= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102517 location_3 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/S1MxHV/NpEw= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102517 location_3 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/YNQW5LXB9U4= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102517 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/OstoFtBpDC8= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102517 location_1 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/UiofebRSeVU= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102517 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/jBSQORU8qi8= AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102517 location_4 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/GNQrUXLJPG0= AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102517 location_1 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/9D6Ynbv7ojw= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102517 location_2 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/OxO5Od/XiUA= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102517 location_3 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/XcToaiyzWys= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102517 location_1 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/Zt3VdJ1jR3s= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102518 location_3 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/BuqQ9Yp8IUo= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102518 location_5 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/gYE7VgxALlE= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102518 location_2 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/VO0lgkZFJCA= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102518 location_4 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/qN7aZv4cdHI= AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102518 location_5 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/jKI8KXjozCU= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102518 location_3 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/vFOwzck6BBM= AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102518 location_4 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/jC1OfZ3i/nk= AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102518 location_4 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/59jsd3D7awU= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102518 location_2 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/pKCq6iEhaFw= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102518 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/duvFAQ/HYic= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102519 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/RR5ESCjWWDs= AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102519 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/ne97UK5trnw= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102519 location_3 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/5kBRgf77Mgw= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102519 location_5 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/Kzl9jqLUMgQ= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102519 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/0jOeJMIdryc= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102519 location_2 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/ELVmm70tETg= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102519 location_4 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/1q9nMvHLTnE= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102519 location_3 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/F8Bf7BH5cmQ= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102519 location_1 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/+df+AHL8jR8= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102519 location_3 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/xRNSdYCOjDQ= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102520 location_3 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/vrwlSupsiDY= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102520 location_3 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/T8CZ6rs/jWY= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102520 location_5 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/5wJ5FCbHhns= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102520 location_4 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/atikuFlf5TY= AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102520 location_3 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/DFBmCX0PyCw= AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102520 location_4 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/4R6+A+qNIS4= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102520 location_3 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/uQBB6qzuTWU= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102520 location_4 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/xd0TY3E+nUI= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102520 location_2 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/O1Q3NA+cjjo= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102520 location_3 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/TP+9g9v/fVo= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102521 location_5 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/Orq+ViiewWo= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102521 location_5 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/CvsVmDhvijw= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102521 location_1 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/e9u4LIR72X8= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102521 location_2 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/OoZQpZERaBs= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102521 location_3 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/o7i8nGUz/Ww= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102521 location_2 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/s5BDe5qH5Gg= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102521 location_1 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/7m7+AIKjkRA= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102521 location_5 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/3tZv8NsRBiQ= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102521 location_4 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/gpyqfMRZHTQ= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102521 location_2 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/7iOlyhnenG0= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102522 location_2 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/FC6Tbd/+Xhg= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102522 location_1 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/3UNXSKWV6Ro= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102522 location_5 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/O8dAsCXIKC0= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102522 location_3 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/1HM1ngEsZkM= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102522 location_5 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/qF755Iht/Uw= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102522 location_2 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/LNQYzoTj2yA= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102522 location_2 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/y0Hqvqbegh8= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102522 location_3 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/TR/2TGk4Qy4= AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102522 location_1 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/mqR9QoLJKCQ= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102522 location_2 product_15 AwEDAAAAAgABAAAAAAAAAP////////9//ZF5ZgQWQjQ= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102523 location_1 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/CBXmkG2HNgk= AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102523 location_3 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/VTvL8WBNSws= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102523 location_1 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/4046GvUFgEc= AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102523 location_2 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/ydW/AH4Pu1U= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102523 location_4 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/gpZ9rVJfsiQ= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102523 location_3 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/4bFkHIK9hU4= AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102523 location_2 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/f12QtvyVWxM= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102523 location_1 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/y1QgNcbiNhA= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102523 location_3 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/ydKtkLsPvhc= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102523 location_5 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/DS/dPzFuT0Y= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102600 location_1 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/m0ietvkFsi4= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102600 location_2 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/Oi4HneX1B0M= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102600 location_4 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/p6QYo+hKank= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102600 location_4 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/NQYR3qcovzU= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102600 location_3 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/Q36YbF0Brwg= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102600 location_3 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/hlMnMMN11Hc= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102600 location_3 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/q1v2wADvlTU= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102600 location_4 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/JCbxT6OntmI= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102600 location_3 product_12 AwEDAAAAAgABAAAAAAAAAP////////9//KG/QeQlOHU= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102600 location_1 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/2ADNGeb/Ljk= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102601 location_5 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/wyawAuXQDiI= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102601 location_5 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/9CbB4ML03Vk= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102601 location_4 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/AupJsLomN3Q= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102601 location_5 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/gmJE36rsLz4= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102601 location_4 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/M+m9sVnmujo= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102601 location_4 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/151P1dI3l3U= AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102601 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/M1s/X66RCRI= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102601 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/CHCpRIzE/BI= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102601 location_5 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/2YulRUQXdUs= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102601 location_5 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/XzjqOuAE1ng= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102602 location_5 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/JYC3ohZvzEQ= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102602 location_5 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/gZzph2x1fXk= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102602 location_5 product_15 AwEDAAAAAgABAAAAAAAAAP////////9//hMB15ATXnU= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102602 location_3 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/3oYSdGXXtCg= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102602 location_3 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/8tJtmzrGZAs= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102602 location_2 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/zlbev9Ew7hU= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102602 location_1 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/GO95JTkLQBg= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102602 location_3 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/+xVPG0P3wwI= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102602 location_5 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/rNYSp5EpWTs= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102602 location_1 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/PFOn6d/7nDk= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102603 location_3 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/zMcLB5GJjUg= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102603 location_5 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/QCMHEdBmDDM= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102603 location_5 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/RCS/aKVvd1I= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102603 location_4 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/qzPfq5w38VQ= AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102603 location_3 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/BpebS5GNEFI= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102603 location_1 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/zNg4l7Ry7QE= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102603 location_4 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/pTobhHORESg= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102603 location_5 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/JbOLfNPPl1w= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102603 location_4 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/0pjjImbp6U0= AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102603 location_3 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/qv3nSgJ+Wlc= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102604 location_4 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/JeF1d3Y9GUc= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102604 location_3 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/3DSePgj8aBM= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102604 location_5 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/qWPi7SkAIWY= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102604 location_4 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/9hTiMqPyEEM= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102604 location_3 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/eg4RxtsLPm0= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102604 location_5 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/OcndxpAYvEA= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102604 location_4 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/mQclEF6C7zM= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102604 location_2 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/sEu5cFlYJzQ= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102604 location_5 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/IKtoamReJRA= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102604 location_5 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/ZOta9ekTIWk= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102605 location_3 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/4Ha+37KKW3s= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102605 location_1 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/xNwsyRYKuwQ= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102605 location_4 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/HA0rCOD303E= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102605 location_4 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/a71dcnJ7SGw= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102605 location_4 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/7Aw05ort+hU= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102605 location_1 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/EtsTQll7CFw= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102605 location_3 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/aJECvwT+cUY= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102605 location_4 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/J+nq0jC2nlw= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102605 location_3 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/GRJwAOVGiWs= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102605 location_3 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/CoHRImz7Yko= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102606 location_4 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/giMdHhndKBA= AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102606 location_1 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/b3oE/gEl6Qg= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102606 location_5 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/zyiUpvxnxyg= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102606 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/OwgGqa1pJwo= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102606 location_2 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/J9bqbKfspw8= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102606 location_3 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/IM/zNxranyM= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102606 location_5 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/5m7bN4JrFU0= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102606 location_2 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/ckRqUtlrX0c= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102606 location_1 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/iF3vI5xq1ms= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102606 location_2 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/kIP5UpMlWBk= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102607 location_1 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/pXKOryGeWxQ= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102607 location_3 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/i4qp2brOEmQ= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102607 location_5 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/g8Kfv4WLFT8= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102607 location_1 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/lfIdjlnl6Sk= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102607 location_3 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/kyRJFGXaaAo= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102607 location_2 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/88IGI8k7pSU= AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102607 location_4 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/S5z1m0RlNS4= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102607 location_4 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/MhrKLZaGYiQ= AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102607 location_1 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/gc8tig3SMT0= AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102607 location_3 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/6R9GwjIoTko= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102608 location_3 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/OOZrbtENtGw= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102608 location_1 product_4 AwEDAAAAAgABAAAAAAAAAP////////9//wrmYkrEIFI= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102608 location_4 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/JlPgJ+8wXEw= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102608 location_1 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/RFeVuSoFGVY= AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102608 location_2 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/2TKmbYuTzBA= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102608 location_1 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/Oy14jqaedCE= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102608 location_1 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/c2ZEj7FN9hY= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102608 location_4 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/xIbv9HfuORY= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102608 location_1 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/JjxYxjxzAlk= AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102608 location_3 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/O4OS07Oe/WY= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102609 location_3 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/yrjs7zWGYQ8= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102609 location_1 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/KQU9RJ9bQX4= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102609 location_1 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/LVA5XN7ZmVg= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102609 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/W/74b8yNYwI= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102609 location_2 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/xm0FWo7/F1Q= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102609 location_1 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/JwCdf3hIhVo= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102609 location_3 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/hqRO1luF1m0= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102609 location_2 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/cmVIej2uEkM= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102609 location_3 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/e/9ULlVvhUs= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102609 location_4 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/jxN/nYMrIQE= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102610 location_5 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/iHm9/hlsWkY= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102610 location_3 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/lnzcMfhGlDQ= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102610 location_2 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/p0VedthdGVM= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102610 location_4 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/SY0/xbA5FhQ= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102610 location_2 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/Y32CtWgnY10= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102610 location_2 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/x3XuZ+6TSjo= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102610 location_5 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/iAGkQbGGNB8= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102610 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/+Oi+7u+BtCM= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102610 location_2 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/NtJNhppFh1Q= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102610 location_5 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/ai0Hzj/1B0g= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102611 location_5 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/rHjZg0Udszw= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102611 location_3 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/xDojGL1uhQg= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102611 location_2 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/CQIoBvQtoms= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102611 location_3 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/R+5buO1r4j0= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102611 location_1 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/LEwK4AF/RAg= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102611 location_3 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/J9HB/EnJ1E0= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102611 location_3 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/P78wXwOMC00= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102611 location_5 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/7aeE350A3RI= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102611 location_5 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/Mi7/kgLuFwg= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102611 location_5 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/fWEo9uNkzW0= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102612 location_5 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/Q6DNk0m7+jo= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102612 location_3 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/qPEICzNY1HM= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102612 location_1 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/CHqdGL+V9SU= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102612 location_1 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/MgqGIkozID4= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102612 location_2 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/k5qFeIJxqDU= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102612 location_1 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/RL/MMZkXnzI= AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102612 location_1 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/x875GnUtdho= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102612 location_1 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/fLtSWbXuPjM= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102612 location_1 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/6a23ifXeOg8= AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102612 location_5 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/8+uDdnGoLVg= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102613 location_1 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/8VV9Ml5hERU= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102613 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/Wv8BvNHYo38= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102613 location_5 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/2HwB2tDmMh0= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102613 location_3 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/CQlsiQ650Qo= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102613 location_3 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/gojtxli8XEg= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102613 location_4 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/PGZAfy8vrD4= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102613 location_5 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/vvASgvaZwwY= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102613 location_5 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/y+rDKsTpGjA= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102613 location_4 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/O9rCrRIk3x0= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102613 location_5 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/2GNzSzMqwBk= AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102614 location_1 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/qc+M67WHPhw= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102614 location_3 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/BDhwDkFCiUo= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102614 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/c3lMaibEsig= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102614 location_4 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/GcZ1GjzWhj0= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102614 location_5 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/VIVmL0MGWSU= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102614 location_2 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/1gb6H5sRyUs= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102614 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/8bEpHXdODHs= AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102614 location_2 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/ocf15X8pIWA= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102614 location_5 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/vyGsOqWM7wo= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102614 location_1 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/I8i7C3gGPyk= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102615 location_2 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/LLls3jIWc0Y= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102615 location_2 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/N2od1SLD9T0= AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102615 location_4 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/p7+1s0lzJgw= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102615 location_3 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/tkuh97gfDD4= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102615 location_4 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/qnZSpcT7Z2I= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102615 location_1 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/qogV5ib/WiU= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102615 location_3 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/qc+M67WHPhw= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102615 location_2 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/QJ/9v7xoGAQ= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102615 location_1 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/pQxP1PISWT8= AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102615 location_1 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/uuUPtSVYjy0= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102616 location_3 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/9K5P0MKcbCQ= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102616 location_4 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/dcyMfBjvUig= AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102616 location_5 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/QummCFQFcHY= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102616 location_4 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/saPM57mJZhQ= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102616 location_3 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/rMLq3zytrgE= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102616 location_3 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/rKweEUnrd3Y= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102616 location_3 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/jIlVMu1t/F4= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102616 location_1 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/c1IG+mNRnl0= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102616 location_1 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/O2FFV5Fq1UU= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102616 location_5 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/Rc7w2Nu+k1I= AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102617 location_3 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/lC9Dskr+AS0= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102617 location_5 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/u6RqRnCjXyI= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102617 location_3 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/zyiUpvxnxyg= AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102617 location_3 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/GfnBsiI7fDY= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102617 location_4 product_5 AwEDAAAAAgABAAAAAAAAAP////////9//LLUwEi6Ohs= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102617 location_2 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/lzis391ZKDM= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102617 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/6Ss2vgZi5BE= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102617 location_5 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/7nIlnkGje3g= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102617 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/WmMrwRlW7hQ= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102617 location_1 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/T2T0F/4yMlA= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102618 location_5 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/3kjlykqJ4Ak= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102618 location_5 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/SpravPTPDUU= AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102618 location_5 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/1t8YdPl3LVs= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102618 location_4 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/1R7XxABFKlQ= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102618 location_2 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/887KxGToq24= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102618 location_1 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/KY3FTxn8Kgc= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102618 location_4 product_12 AwEDAAAAAgABAAAAAAAAAP////////9//J43EG6CYXk= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102618 location_1 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/keaDwGC/11Y= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102618 location_1 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/1uLsNFfsPF8= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102618 location_5 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/KaTZlP2kOAQ= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102619 location_2 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/TR8puauvNjM= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102619 location_2 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/yx3tOKaOinU= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102619 location_2 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/2a6I2HHm6Us= AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102619 location_4 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/eA4dFnePBT4= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102619 location_5 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/c7W4dE6zzm4= AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102619 location_1 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/Wiwu6gukLx4= AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102619 location_3 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/VLpm1wsdLWM= AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102619 location_2 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/RyAUUk/IEB0= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102619 location_1 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/SX/y317+fks= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102619 location_5 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/IKtoamReJRA= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102620 location_4 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/mrBwH+Q3Pi0= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102620 location_5 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/osAY/2SMTRI= AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102620 location_4 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/NGt/07Ml2lM= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102620 location_5 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/+kEqURQsIkc= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102620 location_3 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/gzlyGohXIQg= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102620 location_3 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/vtX1sT9XlCM= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102620 location_1 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/HIBgM+9iYnY= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102620 location_1 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/NSsffQOaTyM= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102620 location_2 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/EZ8Yt9Akm00= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102620 location_1 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/o7KGAvYmpQo= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102621 location_5 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/rVpHB2+pTmE= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102621 location_5 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/r+O1xnbgXSs= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102621 location_2 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/lIZi7ZWtEhg= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102621 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9//qJD2ZABdGE= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102621 location_1 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/QNVnLyeSphE= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102621 location_2 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/WrpnFRSQYUg= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102621 location_2 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/U1Da6Z0gkGo= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102621 location_3 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/pfKyGuWKe08= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102621 location_5 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/NGt/07Ml2lM= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102621 location_2 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/M+r/JDbJAik= AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102622 location_5 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/lO03o11Ubwo= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102622 location_1 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/AA0TOPGO4Cw= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102622 location_4 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/A2g7zCXKWQI= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102622 location_1 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/7nCxmr+TZiE= AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102622 location_5 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/PhfGLesWhh4= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102622 location_3 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/IVOfdV3JG3g= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102622 location_3 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/iAGkQbGGNB8= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102622 location_1 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/KSahUvC8Gx8= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102622 location_4 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/J9tKw1vi3VI= AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102622 location_1 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/nujWe9s912k= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102623 location_4 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/AQ4T1UcK+n0= AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102623 location_1 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/BnrKZM5qLSc= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102623 location_4 product_13 AwEDAAAAAgABAAAAAAAAAP////////9//6kfuMeILUc= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102623 location_3 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/16Ayvne8VgA= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102623 location_5 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/tRizK/bpHQg= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102623 location_2 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/XJY6NewRfkU= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102623 location_3 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/iSvRLikzgGI= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102623 location_3 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/PmpMT6ZTB2U= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102623 location_2 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/Yvjz8d2TLWc= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102623 location_5 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/W4xKx0FRpj0= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102700 location_1 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/5NlDZ77lVBI= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102700 location_2 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/bKpWqSpHSCE= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102700 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/DxXHJsoNhRM= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102700 location_3 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/ml+GX+tO7Fc= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102700 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/tO00Fd2BmnA= AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102700 location_5 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/Anarr7f01yM= AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102700 location_5 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/Y0v7hzuT3mw= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102700 location_1 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/hUs3uQFy6Rw= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102700 location_3 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/eG58GyPQWDQ= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102700 location_2 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/JABPG4KU4Rg= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102701 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/Ne6Y03djAVY= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102701 location_2 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/OWKfreAtNWY= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102701 location_2 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/Gf9Pq/og6zE= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102701 location_1 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/no1mFI+sA1U= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102701 location_1 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/8OlS0Pj2B00= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102701 location_4 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/8JhvjeHRej8= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102701 location_4 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/J5d0bK2pBgg= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102701 location_2 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/pp9FP2yzHmw= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102701 location_2 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/4lN8jVU11yc= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102701 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/uNWsu6GDhms= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102702 location_2 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/QxyCav0I9jI= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102702 location_4 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/uuWGLkP5f34= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102702 location_3 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/O7a77eENkTs= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102702 location_2 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/E9Asaq0CQ2Q= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102702 location_2 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/I1M0SwfGvmE= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102702 location_5 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/B6IduyZ24gY= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102702 location_1 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/KEZNo9faBi8= AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102702 location_3 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/cZ+Z7tWB4jI= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102702 location_2 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/h0JZwasZG0o= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102702 location_1 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/DwRpgKWNkgU= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102703 location_1 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/nXhHrNpCTWQ= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102703 location_3 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/Rsu1H9u0B1Q= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102703 location_5 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/9GmjRz0n+y0= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102703 location_1 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/JIZNMzXLhXE= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102703 location_5 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/1mzkpcO7D1Q= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102703 location_2 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/aNwe6tce4zM= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102703 location_5 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/fH8s5lyGDxM= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102703 location_2 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/xmInuzNYb2E= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102703 location_1 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/bD1Tl0Pa4Do= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102703 location_3 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/SbDON6n1V2Y= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102704 location_4 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/sn2Us599lHQ= AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102704 location_3 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/iSA/DwzLYQQ= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102704 location_5 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/vo7JGTCTXFA= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102704 location_4 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/HNXs20ptexI= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102704 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/7lZ9DfZSpxI= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102704 location_3 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/o03QUrQUphM= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102704 location_2 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/TiWKSbnJXmM= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102704 location_3 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/nIixmg026H8= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102704 location_3 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/HmzwFfeGEX0= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102704 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/UiofebRSeVU= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102705 location_4 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/h/AIYKfH+nw= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102705 location_4 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/6FwJ6QRUWVA= AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102705 location_5 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/NtJNhppFh1Q= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102705 location_5 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/pC077IV9/Tk= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102705 location_1 product_2 AwEDAAAAAgABAAAAAAAAAP////////9//KfE6UIkq0w= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102705 location_5 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/TAXa67NYYA4= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102705 location_2 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/VJgJuxag1jQ= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102705 location_5 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/gXt3qrthrn8= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102705 location_2 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/qP0fVUIjojU= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102705 location_1 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/Ep5bMDuumDM= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102706 location_3 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/uE1IUc3ryTM= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102706 location_2 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/hq9LSvR+wxk= AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102706 location_2 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/pilaoG/Sm2M= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102706 location_3 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/epbB+Wu9hQM= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102706 location_2 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/wXW2SNTHniU= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102706 location_2 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/90jgt44jx3U= AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102706 location_2 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/hEs0QW6T2Hc= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102706 location_3 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/GBafo/Rbn0A= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102706 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/qgY6HbsVY0Y= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102706 location_1 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/akok8vTZYyI= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102707 location_4 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/2MZyRiDtTWo= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102707 location_3 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/9UBlzYwv8xk= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102707 location_3 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/9gR5U6xDKwg= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102707 location_1 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/ac3DRTuowA0= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102707 location_1 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/KSahUvC8Gx8= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102707 location_5 product_12 AwEDAAAAAgABAAAAAAAAAP////////9//NzlARfB8Vc= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102707 location_2 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/ai5GEZY75xk= AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102707 location_2 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/lC5h+RK+TQY= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102707 location_4 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/zt6I1NBfi0E= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102707 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/xStzxl6rBB4= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102708 location_3 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/5XewbAzkV0g= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102708 location_3 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/2gUWdeI7RUc= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102708 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/hIE0dGGNyj0= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102708 location_1 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/aTUesafyyBQ= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102708 location_1 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/oadfngWebgk= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102708 location_2 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/V11Q206ESiY= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102708 location_5 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/J3NhuuDTq0g= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102708 location_3 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/ljEDbnqdpEo= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102708 location_3 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/R+rMB8TYYHA= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102708 location_2 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/MAHJobDwzlk= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102709 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9//+6muSkhb38= AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102709 location_5 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/OCGehDLrW1Q= AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102709 location_4 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/t6dt3+b1r1Y= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102709 location_1 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/2c+clE1GLG8= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102709 location_5 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/GfTZvblwN0g= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102709 location_1 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/wfQ6jyJBJRU= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102709 location_4 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/HNPOJFjAJlU= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102709 location_1 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/oATATWw1lEo= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102709 location_5 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/p7zrROjR8iA= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102709 location_3 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/DHW4xqFI9UE= AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102710 location_5 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/ggzqcBRJXxI= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102710 location_1 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/VDrYTcwlen0= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102710 location_2 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/zCziIFZyQEI= AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102710 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/VOs3pEBmWzU= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102710 location_4 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/ud1fqbgAqiQ= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102710 location_3 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/1v5dL7D0YRA= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102710 location_3 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/qb/9H9f8umQ= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102710 location_1 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/Ta7KhN5LT1k= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102710 location_5 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/B9Ou+DA7nEM= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102710 location_1 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/fA6UcDkPV2o= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102711 location_1 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/qpfXiPWXsUA= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102711 location_3 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/iW13REV5qRQ= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102711 location_1 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/LSi1RwS6q0k= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102711 location_3 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/N87bqszneTM= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102711 location_2 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/9AH6xngv/W8= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102711 location_2 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/d/aScbzmjmg= AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102711 location_4 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/nA8d227mB18= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102711 location_3 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/S1MxHV/NpEw= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102711 location_4 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/AiyZDMJYQVQ= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102711 location_1 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/llGotcSR+U0= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102712 location_5 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/9zRriconhG4= AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102712 location_1 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/k0utHcOBRHA= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102712 location_1 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/f3khJcln33Q= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102712 location_3 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/KJYrx6zxIDM= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102712 location_4 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/uT7SN+U4Kno= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102712 location_3 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/HVVGh9VAfXk= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102712 location_1 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/YiN4Kk1OZ3U= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102712 location_5 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/hFFQ8lf5DC4= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102712 location_1 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/Fiq8HFYutEo= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102712 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/42HTi+pKW28= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102713 location_3 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/dMn3Fo8FCVY= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102713 location_2 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/9cw/wJKaADs= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102713 location_4 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/hcl5W0xfXxc= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102713 location_5 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/pD1uhKx0AFY= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102713 location_2 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/Ot0Q4l9tFlM= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102713 location_3 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/mkIo01tnpTg= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102713 location_5 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/Gh40MdJXTVU= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102713 location_3 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/bgP80A+5Jns= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102713 location_4 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/6W4CsUGT30Q= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102713 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/SadbqCZbQzk= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102714 location_4 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/pHhu5Ag8pDo= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102714 location_4 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/cyN7nHvP6Rk= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102714 location_4 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/JQBy5dFxBk0= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102714 location_1 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/rTXZkIoKfms= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102714 location_2 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/J1qhJvRoajg= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102714 location_4 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/bWG+PSrabks= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102714 location_5 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/RgRLY2I0ES8= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102714 location_3 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/cqSNrb0aBF0= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102714 location_3 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/DzumUAxFTGk= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102714 location_1 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/gVn4zim/qGc= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102715 location_1 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/qJQ2zv9AMUM= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102715 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/Fm5JzNZLl3w= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102715 location_2 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/o5sj2rVMa1c= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102715 location_3 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/90T3kD5DgiQ= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102715 location_1 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/3LSHMemSPAM= AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102715 location_5 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/baI3ZVQx7Qs= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102715 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/d6TjdLZjFxQ= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102715 location_1 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/2K7pZn6Zcic= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102715 location_5 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/jdMFuZuyyxU= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102715 location_5 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/2wWFBnqBWgs= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102716 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/zm687FtRN2o= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102716 location_4 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/TWeRae9Fey4= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102716 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/FGWncrPqBlc= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102716 location_1 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/R6R2+zdGkVU= AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102716 location_2 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/xm0FWo7/F1Q= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102716 location_4 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/9ddllwNAZUQ= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102716 location_3 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/zIr4bc03gzc= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102716 location_1 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/39U4oADash0= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102716 location_3 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/b963SeW9N3U= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102716 location_3 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/EsRJc4kJ5nc= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102717 location_4 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/S6k+1MPO63s= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102717 location_2 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/PbBD5jBHDSE= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102717 location_4 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/NFi2mjBkSCw= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102717 location_3 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/2cemv8lXm1g= AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102717 location_5 product_13 AwEDAAAAAgABAAAAAAAAAP////////9//B9Jww9seDg= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102717 location_5 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/fZlOHKHf7Ag= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102717 location_3 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/PlkOs8DBNQI= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102717 location_2 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/DRTd9+ztiE0= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102717 location_1 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/rh8AHFJHE2Q= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102717 location_1 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/KltSyytCbQQ= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102718 location_3 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/P3sfi2oS1hQ= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102718 location_1 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/j1wlVLuyVVo= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102718 location_4 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/vdI0ENSeWHs= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102718 location_3 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/lPyPPfDkcXg= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102718 location_2 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/R0pM4st8b3o= AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102718 location_2 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/GA8B/0YNCHY= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102718 location_2 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/n5dRO8+SgB4= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102718 location_4 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/RL1uyuCA93c= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102718 location_4 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/6JYeTL86KVU= AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102718 location_4 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/X4qtgwTndSg= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102719 location_3 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/bSxC9SR8ImU= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102719 location_5 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/W9uyygEJKlE= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102719 location_5 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/tC5U5vfM5To= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102719 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/7DSYNDUYPXw= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102719 location_3 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/sUyaQxFX4H8= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102719 location_2 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/75lRatRzYSY= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102719 location_3 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/ij99Yxw4xgs= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102719 location_3 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/XafQ4hQDMk4= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102719 location_1 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/R2LeacFb+XQ= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102719 location_3 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/oxv/xHUDcHk= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102720 location_3 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/aTUesafyyBQ= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102720 location_1 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/TfK8qDKCwg8= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102720 location_3 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/KPgx/lcTiis= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102720 location_3 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/+iMVv8u1cQQ= AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102720 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/Q75u5Gr7+Gg= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102720 location_3 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/aYQxoYxYUig= AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102720 location_3 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/TqqwPjqo+io= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102720 location_5 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/rZwPDNXQ2Cc= AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102720 location_2 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/6PMrcb5G6BU= AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102720 location_3 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/ShAChC7JGB8= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102721 location_5 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/9oj4iyH12Dw= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102721 location_1 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/gt96omix+jI= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102721 location_3 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/qWHaNA+kbVw= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102721 location_2 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/ujpDS0MeoR4= AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102721 location_1 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/rN6R1hSjH2Y= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102721 location_5 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/THyx6a0p7Ag= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102721 location_4 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/15ZjMyNO2A0= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102721 location_4 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/j8190fZefHM= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102721 location_1 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/01s1ty6NHmA= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102721 location_2 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/KSHF/cEiOXM= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102722 location_1 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/ofEbcaELdEQ= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102722 location_1 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/8ahnPVJ0xEM= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102722 location_4 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/gvkDv8fteHU= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102722 location_5 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/lHVYIyy6LnQ= AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102722 location_3 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/tMbSHBoVA0I= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102722 location_5 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/f80oc2myEw8= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102722 location_2 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/McJXSO8OlwM= AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102722 location_3 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/YbrEPR53tzY= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102722 location_1 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/ZckOPnboODg= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102722 location_2 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/tm/3IIVUBkw= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102723 location_5 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/uuWGLkP5f34= AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102723 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/4qctwshZtUM= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102723 location_3 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/DntyHqhbxWY= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102723 location_2 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/AUFY+BaG7Gk= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102723 location_2 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/OCOHv0Gojyo= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102723 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/OyamecZfC2M= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102723 location_5 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/QEjV9mwblis= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102723 location_2 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/1Jq2mQub43g= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102723 location_3 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/yOxXIaHbG1k= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102723 location_3 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/F9KDF7PpJn0= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102800 location_2 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/Nwfc68q2LSw= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102800 location_4 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/gtpOENxNNS4= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102800 location_2 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/Id+XZFDzbFI= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102800 location_2 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/uytl7ecA6Hw= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102800 location_4 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/jD4CuScrXis= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102800 location_1 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/pCYvm9RT6R4= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102800 location_5 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/F7y7u0nO4ic= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102800 location_1 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/4e82PdZNJjs= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102800 location_5 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/PX5jsh4ptzU= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102800 location_2 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/aVe0qHXhtms= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102801 location_4 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/r0ybKSzRam4= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102801 location_4 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/6n1iJWvZT0I= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102801 location_5 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/w8T6sByD0CY= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102801 location_5 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/42arDk0cMGo= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102801 location_1 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/gd4HoKdwGh0= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102801 location_2 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/J8epRTpaKxY= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102801 location_2 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/zNeGzeavjg4= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102801 location_3 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/IwDRBVPT908= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102801 location_3 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/hUHJ68Tyfj0= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102801 location_3 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/fJguLPHVZBM= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102802 location_2 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/AyzgQfqPi0c= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102802 location_2 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/e1lB4dimWS4= AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102802 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/t7i1PnEESS0= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102802 location_5 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/DFtSegD6/RM= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102802 location_2 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/hVUeBbJOwyY= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102802 location_1 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/xkVAz/2iHB8= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102802 location_5 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/eD8u4QXZJ1k= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102802 location_1 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/GUBwKm0YiC8= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102802 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/qx1d5NHqVBk= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102802 location_4 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/jjYyq0zvCA8= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102803 location_5 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/mYRVReeI1B4= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102803 location_3 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/FOtSMsoH6gg= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102803 location_1 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/0WQFYhoSXD8= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102803 location_2 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/CZuuOtN8WC4= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102803 location_4 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/TfY5Ov/BfBc= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102803 location_2 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/uhvDM3/RAkQ= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102803 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/5BRyyEzf5xE= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102803 location_3 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/WuAZeQvt/3M= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102803 location_3 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/ToEo4LH0vAg= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102803 location_5 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/K76GvOUmn14= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102804 location_4 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/wGsrdg8OOVU= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102804 location_2 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/zGZOJnwFv0I= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102804 location_5 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/mMXEvr7qeUc= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102804 location_2 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/BMr9D+yaeGI= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102804 location_1 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/r3WTFd0HWBk= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102804 location_1 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/yXyrExaad1s= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102804 location_1 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/NMRdNer/exo= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102804 location_2 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/x6nOI0qpBWI= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102804 location_3 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/3FisV+hJ3yw= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102804 location_2 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/QyfTPEwwe1M= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102805 location_1 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/tVHj53dVLEw= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102805 location_3 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/TD5tu189z2M= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102805 location_5 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/dqn3p5F/Dnc= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102805 location_4 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/lNUqcsSZrTI= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102805 location_2 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/MrFW+i0iUBI= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102805 location_3 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/iJE38P4gIRc= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102805 location_2 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/SeDWZxxxpwQ= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102805 location_2 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/fwilyOsNQEA= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102805 location_3 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/B5gOw/PcUzs= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102805 location_3 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/WUJ7UhSNuVQ= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102806 location_4 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/rppAJHRhFUs= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102806 location_1 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/OC+wnQdUlD8= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102806 location_4 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/Y7DZh4qFGUw= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102806 location_3 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/jC1OfZ3i/nk= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102806 location_3 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/PVqM3AmHPlo= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102806 location_5 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/FGK4VRdzKDM= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102806 location_5 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/PXkAmpUNbEY= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102806 location_3 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/qzPfq5w38VQ= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102806 location_3 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/VX1YjXyr4iw= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102806 location_5 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/a0wRaZUiBm4= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102807 location_5 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/sfQZqsqn5gw= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102807 location_5 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/WsCXYZhyozE= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102807 location_4 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/9oZyR718aXA= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102807 location_2 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/gT4IbHAGGzw= AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102807 location_2 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/8l6hKiKeKyU= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102807 location_1 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/nwUmQPpdVi0= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102807 location_3 product_10 AwEDAAAAAgABAAAAAAAAAP////////9//DdNiTviACY= AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102807 location_2 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/wFvSz+td72c= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102807 location_5 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/RFlmZN7xLRY= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102807 location_2 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/umqjTIzgKz4= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102808 location_4 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/TOZJnPpjzz8= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102808 location_5 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/R6R2+zdGkVU= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102808 location_4 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/On3TG/Ckm0U= AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102808 location_2 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/Vvwyw62ApH4= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102808 location_2 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/LA5C+d76y1I= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102808 location_3 product_4 AwEDAAAAAgABAAAAAAAAAP////////9//O9a7lQQCHY= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102808 location_4 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/L+riW0YGTxo= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102808 location_1 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/WFpd5NYMVVg= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102808 location_4 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/E1Ct8FBUKl4= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102808 location_1 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/pF/dD9zDxz0= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102809 location_5 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/QYqPS0xP318= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102809 location_2 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/9KJedC7KRFs= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102809 location_3 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/DabYPulSvBM= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102809 location_2 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/AuiwC26HQAU= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102809 location_4 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/zzT6806bOnc= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102809 location_1 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/1UowjeMO6HQ= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102809 location_3 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/7jCXjKSzfVI= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102809 location_2 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/j3oaMZe33C8= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102809 location_2 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/yeI9GguJtGY= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102809 location_3 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/oEb9yRB7ZE4= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102810 location_1 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/JfC1TrUG3Es= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102810 location_1 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/vFSTkg07NTE= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102810 location_2 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/tqkqEvY4q30= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102810 location_2 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/DawlgBDiZWE= AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102810 location_4 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/yWjK2BjPESI= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102810 location_4 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/dHjuHGmH+jg= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102810 location_3 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/4znUnNjJGUg= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102810 location_5 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/idhFF1myTyM= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102810 location_4 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/Csh0LuetNSk= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102810 location_2 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/nLVdG4kkJgI= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102811 location_4 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/KEz6sbosPko= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102811 location_3 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/xDDNAhRzChc= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102811 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/p8+SUYX7wQI= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102811 location_4 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/SAU1hsN5/Cg= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102811 location_2 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/X+IZ7obZbhA= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102811 location_3 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/NBXQUrxqk0k= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102811 location_3 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/qMqh7NYVLnM= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102811 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/vdOmtzjjJxE= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102811 location_4 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/3vZpvehd4Vc= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102811 location_1 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/D2tj9JnKCTM= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102812 location_3 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/95ajNy9dgz4= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102812 location_2 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/WIUuMHl4m1M= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102812 location_1 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/YO0UOi6BRg0= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102812 location_5 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/a6asCY4lxCw= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102812 location_5 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/bd7PF8gQ0Sc= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102812 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/MiKc6se4d3c= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102812 location_1 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/2JKK/1MiqUQ= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102812 location_3 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/J4AU2O1sOzo= AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102812 location_4 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/mX3u0a49PQM= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102812 location_1 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/IphTfmCeVXw= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102813 location_5 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/mAJEgjMNwGI= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102813 location_5 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/9Bj0FTSjFxw= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102813 location_4 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/pE0JCininCE= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102813 location_4 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/opx0NPEHrG8= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102813 location_5 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/3qeFw6HOPH4= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102813 location_2 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/TkMkrmRUwGo= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102813 location_4 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/PqhijQncszs= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102813 location_3 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/7MgGzsAOkUY= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102813 location_2 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/CIjCnO7Bkis= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102813 location_4 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/IOBsmzkUUFI= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102814 location_5 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/nyIVNYU1+VI= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102814 location_1 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/F9KDF7PpJn0= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102814 location_3 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/NCDW0UJZ5CY= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102814 location_1 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/cyN7nHvP6Rk= AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102814 location_5 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/gmyV2KypsmU= AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102814 location_5 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/KpVNnmaWN3E= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102814 location_4 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/jtpAit/OYBo= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102814 location_3 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/wm+xa7ik5l4= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102814 location_2 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/EDffE2r7x0Y= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102814 location_3 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/01s1ty6NHmA= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102815 location_2 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/a1mAnw8ipAg= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102815 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/OCGehDLrW1Q= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102815 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/ZgVmjXvTPz8= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102815 location_5 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/xjEKMVg30mw= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102815 location_4 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/qViUVmYdOgI= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102815 location_3 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/NyaRpdqf4DA= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102815 location_2 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/VX1YjXyr4iw= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102815 location_5 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/AwUwUELaV1E= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102815 location_3 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/b0bFMOLzPSg= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102815 location_1 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/9XYrqJKfiks= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102816 location_3 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/N1dNDm4JcSY= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102816 location_4 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/Dz27V7OPABg= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102816 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/l7zXpjvMgxs= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102816 location_1 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/mXmPl4vdjXU= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102816 location_2 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/69Ht0SK1ZkQ= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102816 location_3 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/5necI3Us4mE= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102816 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/TaWvXOoQvE0= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102816 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/1LAM2YAZ+gk= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102816 location_5 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/W+fi/QbYgCU= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102816 location_3 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/ZeeYFkkzPTE= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102817 location_1 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/LCFwAdyYZCs= AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102817 location_2 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/62fAGVe76gE= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102817 location_4 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/rN6R1hSjH2Y= AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102817 location_3 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/jC1OfZ3i/nk= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102817 location_1 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/DLrCzClwvG8= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102817 location_1 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/knPYdh2CDFw= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102817 location_5 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/7yQwQ9rKVEU= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102817 location_1 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/hCr78B2v62E= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102817 location_2 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/B6YaohPXADg= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102817 location_2 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/VJgJuxag1jQ= AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102818 location_4 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/Q0puVdD5nzA= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102818 location_4 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/aw7HuD1+i2Q= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102818 location_2 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/x1AjEtE64hc= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102818 location_5 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/CHqdGL+V9SU= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102818 location_2 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/aTUesafyyBQ= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102818 location_4 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/AZX/Yhqc1Vk= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102818 location_3 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/+JoyuU4l5ik= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102818 location_5 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/MPX2xd4uFmY= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102818 location_3 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/vmot1GqCiwk= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102818 location_3 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/7XHZrR3IzBY= AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102819 location_3 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/795B5uZ8dEA= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102819 location_5 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/QTgkCSuXGEs= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102819 location_3 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/S0HHjbBk+mU= AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102819 location_2 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/R4z8J3s9FVU= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102819 location_5 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/pNyIBEJ2Yh8= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102819 location_1 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/yKH/F9BgDAQ= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102819 location_5 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/EYoG8NEdlnQ= AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102819 location_3 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/sX77l5By6B0= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102819 location_5 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/YzXRRaThIyk= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102819 location_1 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/NyKOQuqlihw= AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102820 location_2 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/Q7wwVYQ/1yc= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102820 location_1 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/Df48nqPdLAM= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102820 location_3 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/eevM+EE7xU4= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102820 location_5 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/N7rDIP5WrDI= AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102820 location_1 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/5xbSfstrCic= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102820 location_5 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/RtQbmmEOtiY= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102820 location_1 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/MlYIqnh2UQI= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102820 location_2 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/EZaUUq611gE= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102820 location_3 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/nqEHouK7C2A= AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102820 location_2 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/qzIM3mARoCo= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102821 location_3 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/B+NDQfvVSwg= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102821 location_5 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/q9E5lETClng= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102821 location_4 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/fxegCKaGbUU= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102821 location_3 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/xIbv9HfuORY= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102821 location_1 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/nc+a3igl0G8= AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102821 location_1 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/MUcU+MOtums= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102821 location_4 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/BJdcXPp4an8= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102821 location_1 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/uwhoXbu3Ryc= AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102821 location_1 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/m654qCtd3XA= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102821 location_3 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/6KPS+aD2mgA= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102822 location_3 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/ioLc94J3uyw= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102822 location_3 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/ziQ/eLw4sjg= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102822 location_1 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/SiKfLv3Y1Ak= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102822 location_3 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/GLGx79wgxEk= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102822 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/5cPvupMhHGQ= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102822 location_3 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/+pRRrxncgmQ= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102822 location_5 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/VnN+N+SHAA0= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102822 location_1 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/uYWBMQhN9hs= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102822 location_4 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/gfZejv/OdUk= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102822 location_2 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/ha7jLBrG62A= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102823 location_5 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/CqBSgWW2iCs= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102823 location_4 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/z5C43FrrSlM= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102823 location_5 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/LRo388j3T1M= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102823 location_5 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/aeDSAzuKvC4= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102823 location_2 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/fuf2DmPqyzY= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102823 location_3 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/OsI6DdtGN2Y= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102823 location_5 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/2NBnmuxTdWU= AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102823 location_5 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/62fAGVe76gE= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102823 location_4 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/HDmMBrCkdWY= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102823 location_5 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/6OBIsQLN7QM= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102900 location_3 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/WjG6L5GPMzM= AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102900 location_3 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/s8eQk24l91M= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102900 location_1 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/X++46lCfayQ= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102900 location_1 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/qvA9t8FkJws= AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102900 location_5 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/7yQwQ9rKVEU= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102900 location_1 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/lpC98IgFmhY= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102900 location_5 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/vB9/YOdFBWk= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102900 location_3 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/MuU29ufLI1c= AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102900 location_1 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/ZiTkghpu/UA= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102900 location_4 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/jZVV/CkaHy8= AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102901 location_4 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/cONFTg+fzEA= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102901 location_2 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/nIQ556/zfDQ= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102901 location_1 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/KgN5AokGIWs= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102901 location_1 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/6GWp39xdcWY= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102901 location_1 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/Y+EsNTfSMhk= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102901 location_5 product_13 AwEDAAAAAgABAAAAAAAAAP////////9//ZF5ZgQWQjQ= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102901 location_5 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/OTjghlF0dHA= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102901 location_4 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/3gdf5XARyjs= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102901 location_5 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/4zejO9zjmgo= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102901 location_2 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/akok8vTZYyI= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102902 location_5 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/AHcKbemN+m4= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102902 location_5 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/7ymxVtgp9GM= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102902 location_1 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/nqEHouK7C2A= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014102902 location_2 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/P1ZO1HHQ/Uk= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102902 location_5 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/X4SE+RaE5VA= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102902 location_1 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/mWFP0ZtxqzY= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102902 location_3 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/b95JUyGrvj4= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102902 location_2 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/C0Wc9veHkwc= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102902 location_1 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/rGrhGeyKtXk= AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014102902 location_1 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/a7p7cZxrClI= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102903 location_1 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/08/nLkuFhBY= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102903 location_5 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/qJQ2zv9AMUM= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102903 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/QFREIa9GxD0= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102903 location_2 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/NRI8kaIAtU0= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102903 location_5 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/VrQGEBpBj0M= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102903 location_2 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/9HbkKBkjVls= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102903 location_3 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/p5VLrMiY7BA= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102903 location_5 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/VqHUivLVHhI= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102903 location_2 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/oNe75qx0vR8= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102903 location_2 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/X+dVfCX6Vzw= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102904 location_4 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/lZX9+1tBoQI= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102904 location_2 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/O+vMqEFUNiw= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102904 location_1 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/I4hu42zSigA= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102904 location_3 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/ASjyWWFvOy8= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102904 location_3 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/1LAM2YAZ+gk= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102904 location_2 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/dKi+4wmQKQg= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102904 location_2 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/1g/GdHUkmAY= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102904 location_2 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/AMzjNfsnolg= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102904 location_5 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/0IGEbmx//RY= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102904 location_3 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/JKmsBDGlpBM= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102905 location_5 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/zXR2EPqujTI= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102905 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/br8VLBogJQ0= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102905 location_2 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/VYkegGtL2S0= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102905 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/tlfRbWF0kHI= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102905 location_3 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/ghQF3jeVtyE= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102905 location_1 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/6/njQbRQi1U= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102905 location_3 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/gMJqGPAJT3I= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102905 location_1 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/tAn1TgZW9iQ= AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102905 location_5 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/YK6VIkLJyF0= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102905 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/Mec6xz4pxSQ= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102906 location_4 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/Of/e3XF8LCU= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102906 location_2 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/s/BWbzMUIA4= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102906 location_3 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/htl8oRcGsmk= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102906 location_3 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/0JyImgdbUwU= AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102906 location_5 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/NaAmBHBN70w= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102906 location_5 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/KILo0M3sehI= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102906 location_1 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/5/bOvIujMG0= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102906 location_1 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/fp5KOnXYIBA= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102906 location_2 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/I7kJ1Z7qfwM= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102906 location_3 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/kBnc6RJpLUk= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102907 location_5 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/7doyDQ5COBk= AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102907 location_5 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/D8Wxnd+h80k= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102907 location_2 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/3Ka2YTlhzhQ= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102907 location_5 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/RmW+Wz0bEzU= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102907 location_3 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/+/7hpZmKnnY= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102907 location_4 product_12 AwEDAAAAAgABAAAAAAAAAP////////9//q0hThBT9lw= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102907 location_3 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/byukfGlmPiA= AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102907 location_3 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/EOCQNmlf7yI= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102907 location_3 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/L+hy9ORBym8= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102907 location_3 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/fUHt+TV3fg4= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102908 location_4 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/dFhRLbSL+UI= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102908 location_5 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/mU1yFNXA4X4= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102908 location_2 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/PbTUcvl3a2E= AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102908 location_3 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/8uAbI0ovvmw= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102908 location_3 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/pwdKVt/llkA= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102908 location_1 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/BSrjrL6R1kM= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102908 location_5 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/NAxlauJjSgo= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102908 location_3 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/HyuJVE4MJ3g= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102908 location_3 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/7P+AmUCtK3E= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102908 location_1 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/b95JUyGrvj4= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102909 location_4 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/uf3CbQyOl1M= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102909 location_1 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/95e5Z23H8SI= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102909 location_2 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/qv5pwpl5jjk= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102909 location_1 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/zQUIMlt9lSc= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014102909 location_4 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/z8JBY4XwrC8= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102909 location_2 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/rK3qcbz+Fns= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102909 location_3 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/rthSey6MF3M= AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102909 location_1 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/hys8kzrDNHI= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102909 location_3 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/qHAFihHonBM= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102909 location_3 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/2c+clE1GLG8= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102910 location_2 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/CuZi5SJYK1Q= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102910 location_5 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/fEFt1aQ5Ejg= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102910 location_3 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/XP0ZsK9DMDI= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102910 location_1 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/0/Kbel2PkRI= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102910 location_2 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/qnq2fx44URo= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102910 location_4 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/vCynq8GTcU4= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102910 location_1 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/xrmMVcz2+xc= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102910 location_2 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/Z60AmL0kWQo= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102910 location_1 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/8qcU3WXnED4= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102910 location_1 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/cVnvI1eO3BU= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102911 location_2 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/QNWkjuzRrXE= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102911 location_3 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/v0jj/BFxiG8= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102911 location_4 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/Jcsf2y4Hmjs= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102911 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/y1MngQbDbAY= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102911 location_4 product_12 AwEDAAAAAgABAAAAAAAAAP////////9//04N7+JgOhE= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014102911 location_3 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/0hXjLXTn4VM= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102911 location_5 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/KUMI4HLt7nM= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014102911 location_3 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/uD+BBmBniR4= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102911 location_3 product_15 AwEDAAAAAgABAAAAAAAAAP////////9//KJZPW2t7VI= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102911 location_2 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/aBqD5NojAVM= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102912 location_4 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/NyhQQOPDAHQ= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102912 location_2 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/MKxcc2MD6k8= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102912 location_5 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/JrFbWqhMZ1Y= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014102912 location_3 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/OxCPHgV+WTQ= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102912 location_4 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/J8LPtKYpJQk= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102912 location_4 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/CU8I3k2jVSE= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102912 location_2 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/XE5MOcpxmzk= AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102912 location_1 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/GbcmjWKlKi8= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102912 location_4 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/EoPJgwEu+nk= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102912 location_5 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/M6YsbveQ/QQ= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102913 location_1 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/VXQA8YWciyg= AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102913 location_3 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/ono/hmhh4Bw= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102913 location_2 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/5FgARfIUyhM= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102913 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/HElaNDDxcxI= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102913 location_1 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/C0dieYkRomc= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102913 location_3 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/gWmErUBFpRo= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102913 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/G9xY3zYwbXY= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102913 location_1 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/R6R2+zdGkVU= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102913 location_3 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/gYJH/O9/kVU= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102913 location_1 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/10e/Xbimpng= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014102914 location_4 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/pAd9Zm2ZICQ= AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014102914 location_1 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/+c9k8r8yR1c= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102914 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/LpuZQHtKzUE= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102914 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/Fx47GsWRWBM= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102914 location_5 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/hE+u13vRNCs= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102914 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/iQ6TyI9hCGc= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102914 location_4 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/mrTkeKSnvgw= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102914 location_2 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/rUhLLKkNgko= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102914 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/P1ZO1HHQ/Uk= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102914 location_3 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/gt2hN2gAsj0= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102915 location_5 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/ZQyJidkaXUI= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102915 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/2zu+L3xMEE8= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102915 location_4 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/YQbIHgGz8lI= AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102915 location_2 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/tXoZ6+Hh7S4= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102915 location_2 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/jvXn9dBI0WQ= AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102915 location_2 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/KoFZHj+qFnU= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102915 location_5 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/dpT7WNc95Ro= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014102915 location_5 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/xPxHqPpEeVA= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102915 location_1 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/OGQQoFjLUHM= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102915 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/pd+drNed5HA= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102916 location_2 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/Rbxf9E8Pgz8= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102916 location_3 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/FRGZ5pf9xmI= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102916 location_1 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/85YFeu/l9Ds= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014102916 location_5 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/VXC8cWMmvUU= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102916 location_3 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/Wm/GhZhgTxo= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102916 location_5 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/wcFhjjkrRyg= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102916 location_2 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/AECnlTBl5Wg= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014102916 location_1 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/wNe7WGgRRkU= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102916 location_2 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/9XTs/Un8Qz4= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102916 location_5 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/b3XoUCWLPGc= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102917 location_3 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/SGR3V5/QElQ= AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014102917 location_4 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/qUWANymfJ2o= AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102917 location_2 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/f6SEWDwU32A= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102917 location_3 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/rthSey6MF3M= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102917 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/WJXV5uR45To= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102917 location_2 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/OZRR1uJIEEY= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102917 location_1 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/tNYlx+7OJQs= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014102917 location_4 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/gc6JLzmZ4k8= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014102917 location_1 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/frnEXNDb6zE= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102917 location_1 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/H0ijvPz6+3E= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014102918 location_3 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/88J1UQe8z1M= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014102918 location_5 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/HpBmQXc5rQM= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102918 location_1 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/F/HPgLjfIn4= AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014102918 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/pkrR6cKN9EY= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102918 location_1 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/YT6SikvL4HA= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014102918 location_2 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/axs4wgrrKRU= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102918 location_1 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/z44RAMf3/B0= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102918 location_1 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/P7K+KP9tqgc= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102918 location_2 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/5hqt7zXz/SA= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014102918 location_3 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/BwanQEmaZRo= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102919 location_5 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/pmZRW8JdCm8= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014102919 location_1 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/cYXmF5b6jCI= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102919 location_3 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/RfLCdejMSn0= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102919 location_3 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/OZRR1uJIEEY= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102919 location_3 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/8ahnPVJ0xEM= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014102919 location_2 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/AwUwUELaV1E= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102919 location_3 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/9AF05Vc3rz0= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102919 location_1 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/QNVnLyeSphE= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102919 location_2 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/NScgoN9H4mE= AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014102919 location_5 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/EaKpmZW08zs= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102920 location_2 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/El0dqmzdmTc= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102920 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/Lt733x/KohI= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102920 location_5 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/sB1Ao7Y7KEo= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014102920 location_5 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/HzXacbeyQUM= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014102920 location_1 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/v9Rg5YnF8Gg= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014102920 location_2 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/xpysWMosfTQ= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014102920 location_3 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/nwQcFZr4TXE= AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014102920 location_2 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/CaIezQGa6wQ= AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014102920 location_2 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/2lJQWT+5/CM= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014102920 location_1 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/vHGySdxmjDc= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102921 location_4 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/hFFQ8lf5DC4= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014102921 location_1 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/K8Fkspid8g4= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014102921 location_1 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/paXJtxVi4Uc= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014102921 location_1 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/k5AvA6aDpmQ= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102921 location_2 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/WAJlexaIyBk= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014102921 location_2 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/eb9Kfy8/7lE= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102921 location_5 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/9eU1ECNI7zY= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014102921 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/y3+YuPGczis= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014102921 location_2 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/KfhkHkEUB1M= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014102921 location_2 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/Y/uCCFWJxxE= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014102922 location_3 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/Lq2kc0mTjRA= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014102922 location_3 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/CGn9g+Gsdn0= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102922 location_4 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/d0fY+9Srlkg= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014102922 location_3 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/0AaRo2oN2go= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102922 location_4 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/Ida+ByaBTAk= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014102922 location_2 product_15 AwEDAAAAAgABAAAAAAAAAP////////9//HOtbP3LWHQ= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102922 location_5 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/x2WneesfFVo= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014102922 location_1 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/9k5qUz/Mf1I= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102922 location_3 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/00nr8ErvhGY= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014102922 location_3 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/aR1x2nwL+Vs= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014102923 location_5 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/icEFT2H0a20= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014102923 location_4 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/wxfT3DLBtiw= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014102923 location_4 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/rMLq3zytrgE= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014102923 location_4 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/eRJvgMM7i1o= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102923 location_1 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/90+glfIqCV0= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014102923 location_4 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/vFOwzck6BBM= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102923 location_2 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/ubrev+OLtnA= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014102923 location_3 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/5TohxTRpLEA= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014102923 location_2 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/3h/XzkkR0gg= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014102923 location_4 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/jcBBG/H3WwU= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014103000 location_3 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/3eBhEqpMVQ0= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014103000 location_5 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/DtmqoPMBJQg= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014103000 location_1 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/hCr78B2v62E= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014103000 location_2 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/b3xcUFkPFGU= AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014103000 location_5 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/+kmIRSUdnng= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014103000 location_1 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/iz9LPZDxkzU= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014103000 location_3 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/yf71Io9Tdg8= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014103000 location_3 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/x1CnnN+4Snk= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014103000 location_2 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/wR8HrwBi7Qg= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014103000 location_1 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/E/MSzt35HjU= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014103001 location_3 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/ZxOI63aKO1c= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014103001 location_5 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/Yvb9TgrDOXk= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014103001 location_4 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/m/mpAc+vrkI= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014103001 location_5 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/m6ns9hMMgyo= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014103001 location_4 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/L6+TxsZQE0s= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014103001 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/DjxZx5g6d0w= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014103001 location_3 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/rDePr4C9/FM= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014103001 location_4 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/0rO041jy4x4= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014103001 location_3 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/iP5UXOkiN0o= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014103001 location_3 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/BeoSFlg4Rm4= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014103002 location_2 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/m+N3vXPukik= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014103002 location_3 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/Ry0L00pD9Ww= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014103002 location_1 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/QOX0j73/oxg= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014103002 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/lQdgmb6FJSg= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014103002 location_1 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/9QKTMAz6LiE= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014103002 location_1 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/6lMgDmClOzo= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014103002 location_1 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/KO3K1SCRjyA= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014103002 location_3 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/pJFDZG3RTUw= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014103002 location_1 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/91NIDyj5kwc= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014103002 location_3 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/+B8v/rgPBC8= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014103003 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/WLSXAPgs4Hg= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014103003 location_2 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/kIl5NfwhMQo= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014103003 location_1 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/UwO4yDBvVxA= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014103003 location_4 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/NdCXrARHMQc= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014103003 location_2 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/DWZeVYKdARw= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014103003 location_3 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/oHFg8/39KRk= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014103003 location_2 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/jFBsZBkY+S8= AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014103003 location_4 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/eUL/RxtIKGA= AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014103003 location_2 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/11UcNI0BjjU= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014103003 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/iRKuQiXO9EI= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014103004 location_1 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/UJD+K8m5DD8= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014103004 location_3 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/ct/UzZrdjFA= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014103004 location_5 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/mHXglwjp9nM= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014103004 location_3 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/U6i6oqdCijI= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014103004 location_3 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/aDEWmAym53o= AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014103004 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/D9gch9SEYGk= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014103004 location_3 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/dpwpwK5JYRE= AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014103004 location_5 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/cdtu/U62bHY= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014103004 location_5 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/8RJDrFP5BUk= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014103004 location_1 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/KcCCdwIk9kA= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014103005 location_2 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/S5yGd7ik+ls= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014103005 location_5 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/eS/aiqq/dyI= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014103005 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/OqVS6+lPPSA= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014103005 location_3 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/mNv7iHDmBFs= AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014103005 location_3 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/ciCHYm2chWw= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014103005 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/0fN+LPxFo1Q= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014103005 location_4 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/QsO+/YeWeF8= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014103005 location_5 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/r7r6heHdlUo= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014103005 location_3 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/4tk3Q5a4VRw= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014103005 location_1 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/h+irSGgam0U= AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014103006 location_3 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/8vUcLR6K7Qs= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014103006 location_4 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/vDlJjJk3MnM= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014103006 location_3 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/RRzfCKL4hXQ= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014103006 location_2 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/iwHmvgKHSgA= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014103006 location_4 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/4eh2qvTcBkE= AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014103006 location_3 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/hiNYj5m3Vnw= AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014103006 location_3 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/NyKOQuqlihw= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014103006 location_3 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/FBJKRYdAXSk= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014103006 location_5 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/3NdkX6DO/0g= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014103006 location_3 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/DOfnt7wDijw= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014103007 location_4 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/8eCTAIydqCg= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014103007 location_5 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/HehRLbqMgV0= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014103007 location_2 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/IyH8KtUP1AU= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014103007 location_5 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/jFOJ8CwiNQM= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014103007 location_3 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/rzWYh726mhI= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014103007 location_4 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/TcmodEIZ+xQ= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014103007 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/OPoiNZubL18= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014103007 location_3 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/1vlsqse9SWY= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014103007 location_5 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/43yLfitiqCc= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014103007 location_3 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/ZabBS9wV/0U= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014103008 location_3 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/vfYn3/immiA= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014103008 location_5 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/ydmkYcLuiUw= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014103008 location_2 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/K/z5R8WX9Vg= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014103008 location_1 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/jNA50vo4hG8= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014103008 location_1 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/D4eOKevszAA= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014103008 location_3 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/KI/WyzaAXnI= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014103008 location_5 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/pURXD1KL/hE= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014103008 location_2 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/MIH2jJi76nU= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014103008 location_2 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/kli/NlcsOHY= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014103008 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/60FOl9FZdis= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014103009 location_4 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/nke98N+zjHc= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014103009 location_1 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/FLALtAighFQ= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014103009 location_3 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/Y7BvMBNpZkY= AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014103009 location_5 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/RH21+r8KeDo= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014103009 location_4 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/KxjaERFGXlQ= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014103009 location_2 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/ZbyFYDtL434= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014103009 location_5 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/ZMTu93jbr2o= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014103009 location_4 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/315hIn7P1Ew= AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014103009 location_5 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/mIvPsBRg5Ds= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014103009 location_1 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/nYqWjYQOXWw= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014103010 location_1 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/CRGPWK35LFM= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014103010 location_4 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/yK5snZdgeE0= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014103010 location_3 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/14yQnVuMKTg= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014103010 location_1 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/xp7KSBprnE0= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014103010 location_3 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/KYEhQ0FtZ2E= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014103010 location_1 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/u+q9PiiPYD0= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014103010 location_4 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/fbfEU0DHLTE= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014103010 location_1 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/ymggtAfYy04= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014103010 location_2 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/W0WBrfQZLhQ= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014103010 location_3 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/VujnxauWJkw= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014103011 location_4 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/mOmH4Hk5LX0= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014103011 location_3 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/nRVfaxWv6Gs= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014103011 location_2 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/QxyCav0I9jI= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014103011 location_2 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/taCiuNVS8SI= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014103011 location_1 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/CiktFzZ5Y3c= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014103011 location_3 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/SSkWvfEvixc= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014103011 location_1 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/Cn7rvoalJzk= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014103011 location_1 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/EdQomAbtNGY= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014103011 location_3 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/uwhoXbu3Ryc= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014103011 location_1 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/nWPIuXFr8gg= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014103012 location_2 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/WZ4E+dtj2VM= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014103012 location_3 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/SpblYgi6kXQ= AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014103012 location_4 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/o5nOSbluC3c= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014103012 location_1 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/K6In6NBvASY= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014103012 location_2 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/++dd/X/uTVs= AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014103012 location_4 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/sugQssbanzY= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014103012 location_1 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/7jCXjKSzfVI= AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014103012 location_1 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/rX1VJTN7nEY= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014103012 location_2 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/QttXDSrWHDk= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014103012 location_4 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/TZSL4yMxdXw= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014103013 location_5 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/tT8votdbigI= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014103013 location_3 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/MnsMvcLjn1M= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014103013 location_1 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/zbt/NSOttR8= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014103013 location_5 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/IG+T84wVnTg= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014103013 location_4 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/5BpanAU7EUg= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014103013 location_4 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/QWTcraTOQEI= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014103013 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/lQAyOpiBaz4= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014103013 location_4 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/6Ai0BH0PkG0= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014103013 location_1 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/q3BHxMGBdUo= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014103013 location_5 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/Kf5P8h/ZMTk= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014103014 location_1 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/A4D46RBWkxw= AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014103014 location_2 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/gR3ME/wlLlU= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014103014 location_1 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/JadPGRrQ7mM= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014103014 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/XCvOpti3eEU= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014103014 location_5 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/prZq6dKn/BU= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014103014 location_5 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/p5aTkGlpsGc= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014103014 location_5 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/YaLlCMeidzo= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014103014 location_2 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/0svsqcr9zAo= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014103014 location_3 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/BseSel04Xm8= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014103014 location_2 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/inPQCEJJoSk= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014103015 location_3 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/oQbZPHnWoUM= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014103015 location_2 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/0LAxK0o4GU0= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014103015 location_5 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/rXXl5OpHwR0= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014103015 location_3 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/2ODNaRPOAng= AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014103015 location_3 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/+6h794L9VR8= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014103015 location_3 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/msVGRSbIjCY= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014103015 location_3 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/PhfGLesWhh4= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014103015 location_5 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/IogzFiyi2wA= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014103015 location_1 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/4zejO9zjmgo= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014103015 location_3 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/PKjTVh7tNg0= AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014103016 location_1 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/WmMrwRlW7hQ= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014103016 location_2 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/1QdtBZy0gwI= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014103016 location_2 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/4z1fL7vqbSk= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014103016 location_4 product_5 AwEDAAAAAgABAAAAAAAAAP////////9//lGoro7x/Xg= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014103016 location_3 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/oUBD35re+QA= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014103016 location_2 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/nR9iwZFwlQQ= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014103016 location_3 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/w7CWixhQcXA= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014103016 location_5 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/OJ1vejkeYF8= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014103016 location_2 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/NCOOUMtFKnU= AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014103016 location_2 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/pf/2guNXhDw= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014103017 location_2 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/uW9fQbTKom4= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014103017 location_1 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/n1/ShfIvAjY= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014103017 location_3 product_2 AwEDAAAAAgABAAAAAAAAAP////////9//41J3Rsffmw= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014103017 location_1 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/BhLqXPm9Dxc= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014103017 location_4 product_2 AwEDAAAAAgABAAAAAAAAAP////////9//A90YB2tiX0= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014103017 location_2 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/9n/Lj2bvS14= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014103017 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/MOHYU96cf3A= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014103017 location_4 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/2ex9v6+HgSY= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014103017 location_5 product_2 AwEDAAAAAgABAAAAAAAAAP////////9//f/9lNvuKEI= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014103017 location_4 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/c42P9Wk+tUU= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014103018 location_1 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/QYqPS0xP318= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014103018 location_2 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/rjv76Gs9Nh4= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014103018 location_3 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/dpwpwK5JYRE= AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014103018 location_3 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/W7VxO0E9C28= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014103018 location_2 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/AQY8fxNiABI= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014103018 location_1 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/umNGawcTVk8= AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014103018 location_3 product_13 AwEDAAAAAgABAAAAAAAAAP////////9//3Vy4laswlU= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014103018 location_3 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/bCGdCtF+IHU= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014103018 location_3 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/pidp/G3aKHw= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014103018 location_5 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/qhB3WPNK1gE= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014103019 location_4 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/zIr4bc03gzc= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014103019 location_3 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/u/HXEOi9UGY= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014103019 location_2 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/h+irSGgam0U= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014103019 location_3 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/qwdSW75D+lE= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014103019 location_2 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/YiN4Kk1OZ3U= AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014103019 location_2 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/sVi2Crsh5iQ= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014103019 location_1 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/s2jvO4qDkW4= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014103019 location_5 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/ml+GX+tO7Fc= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014103019 location_2 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/C0Wc9veHkwc= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014103019 location_1 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/KshVpyu1e3I= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014103020 location_2 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/zgKQO67/CA4= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014103020 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/SM1A54jF/ls= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014103020 location_2 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/+vUj1l/JCz0= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014103020 location_1 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/lUZrVsKz7S8= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014103020 location_5 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/AgXHXoefCko= AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014103020 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/DmaNWYus1gI= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014103020 location_4 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/GA8B/0YNCHY= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014103020 location_1 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/XmKkz04Fugg= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014103020 location_4 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/hhTOI/EbOg4= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014103020 location_1 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/gjFlfuaIIhQ= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014103021 location_1 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/difz4OuSmmw= AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014103021 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/5Wv5lHyM2Bc= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014103021 location_5 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/ZabBS9wV/0U= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014103021 location_1 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/u9ch/FP1XT8= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014103021 location_5 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/PTxw9rcxkn0= AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014103021 location_2 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/sUEi/CuAbTg= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014103021 location_4 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/+4kozMsIqhQ= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014103021 location_5 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/I0tIhoav2yI= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014103021 location_1 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/XLb+1u0e1l8= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014103021 location_1 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/b59TAa0EcQQ= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014103022 location_5 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/D4eOKevszAA= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014103022 location_1 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/o9YyEBhFWD4= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014103022 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/VdsKh0s0olg= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014103022 location_1 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/VVfJFkGjwww= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014103022 location_3 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/TVduJDRTFDk= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014103022 location_5 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/Zb4KkdFfslA= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014103022 location_5 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/CKgUDNxrUgY= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014103022 location_2 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/eFjcCs81o0Q= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014103022 location_5 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/1ay9x5iYjEQ= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014103022 location_4 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/XE8LfcNfAig= AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014103023 location_5 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/+XNL0HBSmWU= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014103023 location_2 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/zwXykll2rlY= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014103023 location_2 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/A04Wkj4tjT0= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014103023 location_3 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/2bsjx1MucyI= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014103023 location_2 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/0hlIU7LMQSc= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014103023 location_4 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/eceYIN13rFg= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014103023 location_2 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/D9V1+W9rtnY= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014103023 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/RuJGb4IxvUY= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014103023 location_2 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/sS3kmueY3EA= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014103023 location_5 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/HSRvUvZQzlY= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014103100 location_1 product_1 AwEDAAAAAgABAAAAAAAAAP////////9//yi63rVDbg4= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014103100 location_3 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/fgH7qBtDdGs= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014103100 location_3 product_7 AwEDAAAAAgABAAAAAAAAAP////////9//cLSrhFWLDQ= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014103100 location_5 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/22skqX11ejg= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014103100 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/C2MAYHUiGkY= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014103100 location_5 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/FJs2uT9o9EM= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014103100 location_1 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/7tj/eGdqHS0= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014103100 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/Bi10H4biBys= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014103100 location_4 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/83+UFxWjik8= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014103100 location_2 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/KoFZHj+qFnU= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014103101 location_1 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/nBo50qNke1Q= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014103101 location_2 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/C+qqV4nGvxM= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014103101 location_4 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/S5yGd7ik+ls= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014103101 location_4 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/8dtBot9IqzI= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014103101 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/m7D8Vz9rck0= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014103101 location_5 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/8ir9k1j7VT0= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014103101 location_2 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/1q9nMvHLTnE= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014103101 location_2 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/gV90GGUmYCg= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014103101 location_1 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/ypQgoF6jFj8= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014103101 location_5 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/CHenjsWsams= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014103102 location_4 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/ii8FDtRhdTI= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014103102 location_5 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/vu1NAfekLnE= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014103102 location_2 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/tr5nAKQXOl8= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014103102 location_2 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/C2MAYHUiGkY= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014103102 location_2 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/+y8THAZ/2BI= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014103102 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/TQvSCmDHRhk= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014103102 location_2 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/9i10M+NNTjY= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014103102 location_1 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/wZKujh5moBc= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014103102 location_3 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/62HrdsrS7g8= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014103102 location_3 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/18VnIn748Ro= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014103103 location_1 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/l4+c2M0PtmY= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014103103 location_2 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/TFCy38OVviE= AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014103103 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/AkOyQ8rXrTQ= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014103103 location_2 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/6a2Ko6Bpbig= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014103103 location_3 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/xPxHqPpEeVA= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014103103 location_5 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/0lUpgrKdDAQ= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014103103 location_4 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/GFz3Vk1sZjo= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014103103 location_2 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/9RKP/UC29zM= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014103103 location_3 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/NTDy9LwmiAg= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014103103 location_3 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/hECT3gY3imY= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014103104 location_1 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/y7gouNm26wU= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014103104 location_3 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/QNVnLyeSphE= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014103104 location_2 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/Aegdb4Rdsn4= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014103104 location_2 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/UJlFtan/MxE= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014103104 location_3 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/GFz3Vk1sZjo= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014103104 location_4 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/+7keGpyOJzQ= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014103104 location_1 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/9yPZHYynJTM= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014103104 location_3 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/PQfH3tBC8DU= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014103104 location_5 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/Ep4d5Fy5EyI= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014103104 location_2 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/2GTsG9p4jhc= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014103105 location_3 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/aBqD5NojAVM= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014103105 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/P77IL6B9wCE= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014103105 location_5 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/q1v2wADvlTU= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014103105 location_5 product_13 AwEDAAAAAgABAAAAAAAAAP////////9///kKhfqTkUY= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014103105 location_2 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/uFlMoWfw3A4= AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014103105 location_3 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/HsjVxbWWfFA= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014103105 location_3 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/jkvpQ+zycHQ= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014103105 location_4 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/lPc7aSBqez0= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014103105 location_2 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/iEoTT5+4f0U= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014103105 location_1 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/IQHCnEGewio= AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014103106 location_3 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/ZWbW5Musdmc= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014103106 location_5 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/EjP411uAcgc= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014103106 location_3 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/cUYoDUo6Ei4= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014103106 location_1 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/LyEVl6xdGQY= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014103106 location_3 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/E+G1bs4MPBs= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014103106 location_2 product_2 AwEDAAAAAgABAAAAAAAAAP////////9//FmorDEseCI= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014103106 location_3 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/o+8LdxZZ1BU= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014103106 location_1 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/h63ABryNBwA= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014103106 location_4 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/RE4Iy8Mprkw= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014103106 location_2 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/efVuAH+1b2U= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014103107 location_5 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/aWkBPS2rwnY= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014103107 location_4 product_12 AwEDAAAAAgABAAAAAAAAAP////////9//HWVt9nKMTw= AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014103107 location_2 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/JQoNPYOmnj0= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014103107 location_3 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/ioLc94J3uyw= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014103107 location_3 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/NAYoVJlWCHw= AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014103107 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/SeJrPzqaGgU= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014103107 location_2 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/plJnJW7xqBc= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014103107 location_1 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/0bgSAsYfnxM= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014103107 location_5 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/E77Smams8zE= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014103107 location_5 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/j26MDZsjOhA= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014103108 location_4 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/wtsUzn72+mA= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014103108 location_3 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/c3lMaibEsig= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014103108 location_1 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/4+SSOAbEkEU= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014103108 location_2 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/9hBrz/Ry1Ak= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014103108 location_5 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/NhkC8UhtOgg= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014103108 location_2 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/yz+X5UT4VmE= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014103108 location_5 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/MBKZzri9S1Q= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014103108 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/02DAx9vfIhU= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014103108 location_3 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/8qcU3WXnED4= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014103108 location_4 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/n5U+rUI9zSU= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014103109 location_2 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/oWN/0z/AfW0= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014103109 location_1 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/35w6V/vGBhw= AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014103109 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/jQgrSv35q28= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014103109 location_1 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/p3s3O8pQTxk= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014103109 location_3 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/3lpFzrVZPQo= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014103109 location_4 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/htl8oRcGsmk= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014103109 location_2 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/U1Da6Z0gkGo= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014103109 location_1 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/YVbQlOnlfz8= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014103109 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/R6R2+zdGkVU= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014103109 location_5 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/GNQrUXLJPG0= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014103110 location_5 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/JjYbQu8pEn0= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014103110 location_1 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/+P+vy0ZOCBM= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014103110 location_1 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/Jm6k36INpEw= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014103110 location_1 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/lnde+ecH4zY= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014103110 location_1 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/gbkvNku8fTE= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014103110 location_4 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/JRhg7xOfi24= AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014103110 location_2 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/5PIj9mXy/UY= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014103110 location_1 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/tLlbh5Nbewo= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014103110 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/F5QokNnGZ2A= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014103110 location_4 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/GFCyYQDkKkE= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014103111 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/c9gTlsSRhyk= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014103111 location_3 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/j0jA0s/8oEY= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014103111 location_2 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/bGjcwVMDlnY= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014103111 location_3 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/O2uUX/QV5U4= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014103111 location_4 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/itlbFWSJEFc= AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014103111 location_3 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/C0Wc9veHkwc= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014103111 location_3 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/OckfIsKFiSw= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014103111 location_1 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/rAgYC0e21gY= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014103111 location_3 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/6JYeTL86KVU= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014103111 location_1 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/7FpdQNCmQTE= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014103112 location_3 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/nxH4TmaUajA= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014103112 location_1 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/LJquLeRCazI= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014103112 location_2 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/rzWYh726mhI= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014103112 location_5 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/zXR2EPqujTI= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014103112 location_1 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/MPX2xd4uFmY= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014103112 location_5 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/VkU1OOWjLFg= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014103112 location_5 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/Vr1q3uZs8GI= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014103112 location_4 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/hEbyA8utUj0= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014103112 location_2 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/MlDDSzgm2HI= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014103112 location_1 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/e+z4+Ry7uQA= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014103113 location_4 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/Og7PMMoZdSE= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014103113 location_1 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/qfA5HI1rWFU= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014103113 location_1 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/SlWkZzoFigc= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014103113 location_2 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/AGdn0OGI22w= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014103113 location_5 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/JL6ReCIIoDs= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014103113 location_4 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/CgsMzK9XNWY= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014103113 location_3 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/aJuoxrH60gc= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014103113 location_2 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/L19+i80vKGY= AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014103113 location_5 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/AkOyQ8rXrTQ= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014103113 location_1 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/JfmYt8Yp/Xs= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014103114 location_2 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/foMO37NT/S8= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014103114 location_4 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/VvU38IlK3yw= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014103114 location_4 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/p6QYo+hKank= AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014103114 location_1 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/oHFg8/39KRk= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014103114 location_5 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/8tQoHPVlP0Q= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014103114 location_4 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/s/5gNjr5xgA= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014103114 location_3 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/8vOxNAU+2xg= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014103114 location_1 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/rSHYDyw+8gE= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014103114 location_5 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/PkZIMf/tARI= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014103114 location_5 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/HejmwaTH9X0= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014103115 location_3 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/gkalPkvu/SE= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014103115 location_5 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/TVYc9T/SiDg= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014103115 location_5 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/h09rgY/VTnU= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014103115 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9//gytIXaPvig= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014103115 location_2 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/hfL/0hR2K08= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014103115 location_1 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/RZUCUQG9XTI= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014103115 location_1 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/VYkegGtL2S0= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014103115 location_4 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/IVM3FzS7ylE= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014103115 location_3 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/jxAUir8EGEY= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014103115 location_1 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/KT4w2+oiKmU= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014103116 location_5 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/vhqotu0QMSg= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014103116 location_2 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/+3RmZqVLmQI= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014103116 location_3 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/H0ijvPz6+3E= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014103116 location_2 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/tINJRo15wW8= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014103116 location_2 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/oiQ+yy6oPSU= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014103116 location_1 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/MiQrCiGLAwE= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014103116 location_4 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/MQCInzIwIFE= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014103116 location_3 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/Op3Hst3w8Ss= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014103116 location_5 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/Jp76tSGb6QU= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014103116 location_1 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/knHgRoXd8FM= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014103117 location_3 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/Y6LidzNvCUM= AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014103117 location_4 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/VoTyDBCOMQc= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014103117 location_4 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/j4RwHh23WXA= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014103117 location_2 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/6jj/igZFNRk= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014103117 location_5 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/8vBI+62WJC8= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014103117 location_2 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/5e+A7xrtz0Y= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014103117 location_1 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/qfasGZFyaTQ= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014103117 location_3 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/9K5P0MKcbCQ= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014103117 location_1 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/1cn1C1rovmM= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014103117 location_1 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/4frQeC6yTCA= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014103118 location_1 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/0PDub3hAjkM= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014103118 location_4 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/Im2APXA61AY= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014103118 location_3 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/9kA3mAtwejE= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014103118 location_4 product_3 AwEDAAAAAgABAAAAAAAAAP////////9//gU1lyngqWI= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014103118 location_2 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/Rj66iTRizGs= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014103118 location_2 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/ixLHdjIQjTw= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014103118 location_5 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/CHqdGL+V9SU= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014103118 location_5 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/nwbzmmog4Ww= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014103118 location_5 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/yYyH2fI41yo= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014103118 location_3 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/akok8vTZYyI= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014103119 location_1 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/2jGqg4JEQGU= AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014103119 location_1 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/qj6MTzXjQg8= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014103119 location_3 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/MYnPxqwJj1Y= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014103119 location_4 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/OfeVcgvSyB0= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014103119 location_1 product_10 AwEDAAAAAgABAAAAAAAAAP////////9//J43EG6CYXk= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014103119 location_1 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/YKQFk12oHjQ= AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014103119 location_5 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/73JuaeWialE= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014103119 location_5 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/67bLu/XK934= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014103119 location_4 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/6KAxLZKFKgc= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014103119 location_2 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/LAC2BgUkoFY= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014103120 location_3 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/tf8jrY4lWyA= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014103120 location_3 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/vfRQdi3hlyY= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014103120 location_4 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/5oBNBgemCQA= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014103120 location_3 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/HaRzT0IXxl4= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014103120 location_5 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/wfzD6cGG1RM= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014103120 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/EU4zUtCC0W0= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014103120 location_5 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/2sY0huA38HQ= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014103120 location_3 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/v7J1oGkNxgE= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014103120 location_4 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/OSQPyOcNtRA= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014103120 location_5 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/KVy2lFq2CyU= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014103121 location_2 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/8CXQhuqhMEg= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014103121 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/HDrzrLNnLm4= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014103121 location_3 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/AXe2Od+/7Bc= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014103121 location_3 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/1BejEe2s1FQ= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014103121 location_2 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/09YjmlWCVlw= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014103121 location_3 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/A89VJg0DiW8= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014103121 location_2 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/ZbyFYDtL434= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014103121 location_3 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/9R0x7I9JNiE= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014103121 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/q16H6xcL4DA= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014103121 location_1 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/oJc0t/+FHUY= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014103122 location_2 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/X66ZmbQq8gI= AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014103122 location_4 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/5mW9X3A/5w8= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014103122 location_4 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/qlON3RFYDAE= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014103122 location_3 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/+9eWGwxXT0U= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014103122 location_4 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/WiqzZOIj3lI= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014103122 location_5 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/WrpnFRSQYUg= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014103122 location_2 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/jMROFjhVckE= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014103122 location_1 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/eyH0kUvnWlk= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014103122 location_5 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/sYCgxZyFbHw= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014103122 location_3 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/U1Da6Z0gkGo= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014103123 location_1 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/+mbrkmYMrF8= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014103123 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/QbtXn1yT2Tg= AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014103123 location_2 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/LWGXOP+vb3Q= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014103123 location_2 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/mQOP4/QsAHQ= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014103123 location_4 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/Ak+RJ1G0n2c= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014103123 location_2 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/UCfZ21glVn0= AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014103123 location_4 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/EwQ/iGVVmFU= AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014103123 location_5 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/TrbiCfb3GVk= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014103123 location_3 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/GYoQawz1IU0= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014103123 location_2 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/a+Pry6s9phs= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014110100 location_2 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/ZdX9jTN7GCI= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014110100 location_5 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/qa9H+jpJljE= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014110100 location_5 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/mBNbM9ygr2g= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014110100 location_3 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/AQY8fxNiABI= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014110100 location_4 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/Bki6mIctRz4= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014110100 location_3 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/fmuhn/p+9XA= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014110100 location_1 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/9l1ineQypR0= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014110100 location_5 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/QAfnPCJfCzw= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014110100 location_3 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/DXFu1VQXZg0= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014110100 location_1 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/h8/NOW3/hTc= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014110101 location_3 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/vnfLLNVsGnM= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014110101 location_3 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/dDdO3oWbmw4= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014110101 location_2 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/vrwlSupsiDY= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014110101 location_4 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/s4u6NFDTy0g= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014110101 location_3 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/7vhq3RqcTy0= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014110101 location_4 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/MKxcc2MD6k8= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014110101 location_2 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/pvcWHkjP5jM= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014110101 location_4 product_7 AwEDAAAAAgABAAAAAAAAAP////////9//WtMEZIKaG8= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014110101 location_5 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/MlDDSzgm2HI= AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014110101 location_3 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/tr/7dPe2+Ro= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014110102 location_1 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/Z+NenW+hXgI= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014110102 location_3 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/+Pbi91yfgjY= AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014110102 location_4 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/EdQomAbtNGY= AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014110102 location_4 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/v1hUSfBP8wE= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014110102 location_2 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/0n7TofeCjmk= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014110102 location_2 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/yKYmEwsDjGo= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014110102 location_4 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/FtFPvCWMOz4= AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014110102 location_2 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/PGJ6Cd+2HxY= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014110102 location_1 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/SKC90jTvSW4= AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014110102 location_1 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/F6wbbFWmIXI= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014110103 location_5 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/gfZejv/OdUk= AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014110103 location_5 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/kl9uPkdwMVI= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014110103 location_4 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/1/nmvGvPlGA= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014110103 location_4 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/pXF0j+KjM1Q= AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014110103 location_3 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/k9GzTmtGFhI= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014110103 location_2 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/HnaSHxyhrnc= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014110103 location_5 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/RCeL7Rnqagg= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014110103 location_5 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/thQQn6JfPWQ= AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014110103 location_1 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/dK4342O/FTk= AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014110103 location_1 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/wbLFw9aKk1g= AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014110104 location_5 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/75lRatRzYSY= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014110104 location_5 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/WNo26CiaNko= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014110104 location_4 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/kimcZ3ojN0g= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014110104 location_2 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/BdwXpIdjnC0= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014110104 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/Y+gHLXSnog0= AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014110104 location_2 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/upvGSurrHjw= AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014110104 location_4 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/NZDTg+wXNho= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014110104 location_2 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/HLVfDvdrQCg= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014110104 location_1 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/XyV6EbFzjQg= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014110104 location_5 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/7yQwQ9rKVEU= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014110105 location_2 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/L/Vx5kMX4m4= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014110105 location_3 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/znEhjWWJKwI= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014110105 location_1 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/GoqMVbSatUE= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014110105 location_3 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/H0ijvPz6+3E= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014110105 location_4 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/70pUGYNErWE= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014110105 location_4 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/pkqk0uF1oHg= AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014110105 location_3 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/RffO6CNcVgw= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014110105 location_5 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/BGP9WDAZyic= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014110105 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/RMVUwk0/xjQ= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014110105 location_4 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/5fWLa4hjvhA= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014110106 location_2 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/gfZejv/OdUk= AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014110106 location_3 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/pC077IV9/Tk= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014110106 location_3 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/N4u4RMfUogQ= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014110106 location_4 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/gy7ERbUWMxQ= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014110106 location_5 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/AA/7rcbvLhg= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014110106 location_1 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/4SrqqFKpsSI= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014110106 location_2 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/+8NGPtPX+is= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014110106 location_3 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/aSTf/8T1oS4= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014110106 location_4 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/7m8fxslcgAc= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014110106 location_3 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/ba/xvrJOdhU= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014110107 location_5 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/TGBWoBYd0xo= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014110107 location_2 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/bQEA46pNqBY= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014110107 location_3 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/iNBCSjpRRFA= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014110107 location_3 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/2nTqyjb6Dko= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014110107 location_4 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/D7+tSeyL4hU= AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014110107 location_4 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/b9aFdWiQ9zo= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014110107 location_3 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/BWOe31C4hhc= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014110107 location_5 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/4dBXaDd+pxU= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014110107 location_1 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/ZRjaZZ9w+Rg= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014110107 location_2 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/vlVHw3vfXBE= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014110108 location_4 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/cYyuElKV9C8= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014110108 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/e7tZLZ8wlzg= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014110108 location_4 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/YXtdobQ6+XQ= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014110108 location_5 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/fDxiF1Rq2kE= AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014110108 location_3 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/YXfqaON7pyI= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014110108 location_5 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/TbXZutTxC0Q= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014110108 location_4 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/7HSCjc2qzlg= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014110108 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/aDEWmAym53o= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014110108 location_1 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/0Yo4ZTl/DGs= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014110108 location_2 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/VZhudayEyyY= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014110109 location_5 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/8BUXwMOlnVE= AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014110109 location_4 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/qhB3WPNK1gE= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014110109 location_4 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/4WNNxIzm9xc= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014110109 location_5 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/lCxck2LpRQo= AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014110109 location_4 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/vB9/YOdFBWk= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014110109 location_1 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/G/H8/a/8sA0= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014110109 location_5 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/ie0jP05vqV0= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014110109 location_1 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/dErbZZ2HwVw= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014110109 location_1 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/W4xKx0FRpj0= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014110109 location_2 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/3sPB2FrWUF0= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014110110 location_1 product_2 AwEDAAAAAgABAAAAAAAAAP////////9//aph2xxcv1E= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014110110 location_3 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/HvBenZSWBRA= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014110110 location_4 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/l59mfJ5J3RQ= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014110110 location_1 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/9/9kWh2FWSA= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014110110 location_1 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/NSPZucBomzU= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014110110 location_4 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/ZkzfGSwfWCw= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014110110 location_1 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/5h8impkKjwM= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014110110 location_2 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/f6SEWDwU32A= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014110110 location_2 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/6MhAtjvagzg= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014110110 location_5 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/ogo6mKZIoHU= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014110111 location_1 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/uP6gPaGWMjc= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014110111 location_5 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/MtgccINinWc= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014110111 location_3 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/STyJjFE7ynQ= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014110111 location_5 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/bqYwm9cHEm4= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014110111 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/VDnTL3oJPVU= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014110111 location_1 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/MXFce7S7sH4= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014110111 location_2 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/ryxwWc7EXwg= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014110111 location_5 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/d8EKCKmMLVw= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014110111 location_1 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/wwA54wGMRB8= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014110111 location_2 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/EBCLolez2Sc= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014110112 location_3 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/YtwCb0b4EzM= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014110112 location_4 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/li1RsysroAE= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014110112 location_5 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/jTrAwbx6hQU= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014110112 location_4 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/1q9nMvHLTnE= AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014110112 location_1 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/Oafbgdw1iBY= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014110112 location_2 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/iSufUQQCzls= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014110112 location_3 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/MogO+MlHnjQ= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014110112 location_5 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/uuWGLkP5f34= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014110112 location_5 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/zx0+4C3nrFU= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014110112 location_4 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/HzvlirMsGWY= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014110113 location_1 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/V6wj4hLZUy0= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014110113 location_3 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/bUT6aJKROVs= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014110113 location_2 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/8RE5OYA30As= AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014110113 location_1 product_15 AwEDAAAAAgABAAAAAAAAAP////////9//mpfQJ4wUxo= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014110113 location_1 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/VHC0XlCYS1M= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014110113 location_3 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/7XBxSmsn0jQ= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014110113 location_5 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/DQk5IZeuowM= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014110113 location_3 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/T2JVYAIJ+Q8= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014110113 location_5 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/Q+5RQ2IYLi4= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014110113 location_1 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/JFSDHB7Gtho= AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014110114 location_3 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/Kdxi5YTzpn8= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014110114 location_2 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/E69hLOXTaXo= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014110114 location_3 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/5+Cy8yC0gjI= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014110114 location_4 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/uz1e48XIH3o= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014110114 location_3 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/7m8fxslcgAc= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014110114 location_3 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/mmahFmfReRc= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014110114 location_2 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/ZQyJidkaXUI= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014110114 location_4 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/jfWzUgeUQjQ= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014110114 location_3 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/xnxSXqCRfi0= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014110114 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/qIkdKSy06UA= AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014110115 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/FmbcTcVj8g0= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014110115 location_2 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/SP1p1gzeD0k= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014110115 location_3 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/8rsphAPAuQE= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014110115 location_1 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/OUvsK4BdDnY= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014110115 location_3 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/wVoUBEx3LSI= AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014110115 location_1 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/sq+LKKEYxR4= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014110115 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/On3TG/Ckm0U= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014110115 location_1 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/FksU/0QOpWM= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014110115 location_4 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/Y56KyRexvBY= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014110115 location_3 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/KTppuuSw6ig= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014110116 location_3 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/16azOB2FEgk= AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014110116 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/Y7DZh4qFGUw= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014110116 location_1 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/yDgTMT3wBVk= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014110116 location_2 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/+V/BSM9yNEY= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014110116 location_1 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/5QJ1bGuRORg= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014110116 location_1 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/sSLilqnb4n4= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014110116 location_5 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/pS+4lm9Yw0c= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014110116 location_5 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/qN3eyvpkqUo= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014110116 location_1 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/E8nFkXQGpRk= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014110116 location_4 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/ryT/GBlpVDg= AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014110117 location_4 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/jzl7F3F5C1U= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014110117 location_1 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/Gbdp9Bb3Uik= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014110117 location_3 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/xB+eHto6oCM= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014110117 location_2 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/r/nK5YySwk8= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014110117 location_1 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/VHOpXBnCR0o= AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014110117 location_5 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/YKnYsRBYKAE= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014110117 location_3 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/BRGdjC8W0Ts= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014110117 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/ISbqS3Ev1T0= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014110117 location_1 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/Cft9WcDR9z0= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014110117 location_1 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/hCJ7FiStp2A= AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014110118 location_3 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/AskJHT94tDU= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014110118 location_2 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/gt96omix+jI= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014110118 location_1 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/R3DXqM2C3SM= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014110118 location_5 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/liHoutkINxU= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014110118 location_3 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/9gQYngRsgSI= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014110118 location_3 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/1p9muQE1KFw= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014110118 location_3 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/6l7tLROMqVQ= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014110118 location_5 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/fL/7rpt1MHU= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014110118 location_1 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/37Sph4xYWTw= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014110118 location_1 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/Q+5RQ2IYLi4= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014110119 location_4 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/6GSiGKe+U3w= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014110119 location_3 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/auFDF4ib7BM= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014110119 location_3 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/7BsL4rixHRg= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014110119 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/xRhh1NsgGAw= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014110119 location_2 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/8LENYLt4Fh4= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014110119 location_4 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/KI/WyzaAXnI= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014110119 location_4 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/HyuJVE4MJ3g= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014110119 location_5 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/vOCytry+yCw= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014110119 location_2 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/3x045W24GWU= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014110119 location_1 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/3Y9hQq6vhWQ= AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014110120 location_2 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/OU82vcko1RA= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014110120 location_1 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/D5Vg3YTdqVY= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014110120 location_1 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/B9Ou+DA7nEM= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014110120 location_5 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/ILQPrDfFozI= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014110120 location_3 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/JMMm5peCPn0= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014110120 location_3 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/QINsKuqU31E= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014110120 location_4 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/EMAu0ZZE9jU= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014110120 location_2 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/WS3I49rHyCA= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014110120 location_4 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/R9/UETXuuHU= AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014110120 location_3 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/+v8xbYvYU08= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014110121 location_1 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/RR5ESCjWWDs= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014110121 location_2 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/jeH5LPDxWy0= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014110121 location_4 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/p2ceJAp9GV8= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014110121 location_1 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/ZQjPSWI2hhk= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014110121 location_2 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/VsFWMBCmLGM= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014110121 location_2 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/fZDgWkzMDRA= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014110121 location_5 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/rd9aH2v3FlA= AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014110121 location_3 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/RsLFHx1NnEU= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014110121 location_4 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/Pq7WsHomlhk= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014110121 location_4 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/glRFnAegoC4= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014110122 location_1 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/QCw2o4ftLTc= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014110122 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/pilaoG/Sm2M= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014110122 location_5 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/zm687FtRN2o= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014110122 location_5 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/7azpDk/X4kQ= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014110122 location_2 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/YVyZSh+iy00= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014110122 location_4 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/aa/yTcesfw0= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014110122 location_2 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/V2GDp3f2CVQ= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014110122 location_1 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/xa/TR8beTWM= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014110122 location_1 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/QejFoYyhGCs= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014110122 location_4 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/68d9hryMiko= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014110123 location_4 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/PX94pTbAARM= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014110123 location_2 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/Gl598Kb7VDA= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014110123 location_2 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/6LIBiTLTeTA= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014110123 location_2 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/y+zZoJn7GSY= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014110123 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/qzPfq5w38VQ= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014110123 location_1 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/zlN5UWsUpnU= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014110123 location_4 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/eBT3A0HDHhA= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014110123 location_2 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/HD75wFLCfBY= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014110123 location_4 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/wSNeBWfbMxg= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014110123 location_5 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/B8RUZkwetwI= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014110200 location_4 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/dAuQIgsOSjQ= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014110200 location_5 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/0fyWiF/NmXw= AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014110200 location_3 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/TcGNPi74nmU= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014110200 location_4 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/PsIlO1gtuio= AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014110200 location_3 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/3tPMezFm1yc= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014110200 location_1 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/IGMeb4A7Q0w= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014110200 location_1 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/00dYSte8tkE= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014110200 location_2 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/+QmU+Q41pl4= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014110200 location_4 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/IIV/tW4wXmE= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014110200 location_2 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/Qdi5XZJWHSo= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014110201 location_5 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/eUL/RxtIKGA= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014110201 location_2 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/nnRM+bCB+UI= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014110201 location_3 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/KuEALf9UHlg= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014110201 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/VE6E8NKM6mI= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014110201 location_2 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/cGw6sQbrjC0= AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014110201 location_5 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/81bvl5lL7l8= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014110201 location_2 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/T2JVYAIJ+Q8= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014110201 location_1 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/Q/813biBsEw= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014110201 location_5 product_3 AwEDAAAAAgABAAAAAAAAAP////////9//zn6axYQoC4= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014110201 location_2 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/Ea+W8vFnTnA= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014110202 location_1 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/iwibBbn8zmc= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014110202 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/Cq1NNkVvAiA= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014110202 location_1 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/5J1jLgI0fic= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014110202 location_2 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/njX5GgWoLV0= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014110202 location_4 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/cdS/YmNKAgA= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014110202 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/rYHsokh0azM= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014110202 location_5 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/PAO01crhsUQ= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014110202 location_1 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/Og7PMMoZdSE= AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014110202 location_3 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/VUQPDzW7GiY= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014110202 location_2 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/l7xBWSpuHUQ= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014110203 location_3 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/XKBIDJH7mUE= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014110203 location_4 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/6mRS3aa/33E= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014110203 location_1 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/OaJYFiAa8gY= AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014110203 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/oh2XafR0ulY= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014110203 location_5 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/n5U+rUI9zSU= AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014110203 location_2 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/EuA7iasMjBc= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014110203 location_1 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/yrEXUwdjVjc= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014110203 location_1 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/lXd3l8gTBlU= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014110203 location_4 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/9k2gEvJ+gkk= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014110203 location_4 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/PwOY5Wa23AQ= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014110204 location_3 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/PbTUcvl3a2E= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014110204 location_1 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/OC+wnQdUlD8= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014110204 location_4 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/gOmmv39+eWQ= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014110204 location_3 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/I1M0SwfGvmE= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014110204 location_4 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/ESEcRxmuUBk= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014110204 location_4 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/VUNk9pqhBD8= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014110204 location_2 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/DzZUwbzNBjc= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014110204 location_2 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/HC1S48bcwjs= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014110204 location_3 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/4BdZzt5DQFU= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014110204 location_4 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/T/zVEYqqGwk= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014110205 location_1 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/CAzvBbWlnSE= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014110205 location_1 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/z8KEu8Pw+Do= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014110205 location_1 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/f3oLMqAADCI= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014110205 location_2 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/beBWiWgIn24= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014110205 location_1 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/3Y0fvSeEOQE= AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014110205 location_5 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/Jf5Hi1pRdAE= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014110205 location_1 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/qynul6NtjyM= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014110205 location_4 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/Ot2a/0h+mB8= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014110205 location_1 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/7lZ9DfZSpxI= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014110205 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/KCZEeTWPv2U= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014110206 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/3Hw5rd7enX4= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014110206 location_1 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/NgjTgn+9DEE= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014110206 location_4 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/EfrKxU+1lQc= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014110206 location_4 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/m/mpAc+vrkI= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014110206 location_3 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/HaRzT0IXxl4= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014110206 location_1 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/H/bI/Vkp7Aw= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014110206 location_3 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/zXR2EPqujTI= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014110206 location_2 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/6a23ifXeOg8= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014110206 location_4 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/bCwj3bahuxU= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014110206 location_2 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/nw8X7RDLDD0= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014110207 location_3 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/WusSoLuoBzg= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014110207 location_4 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/CUKQdgvlN04= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014110207 location_1 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/M7Rno9BT9Hs= AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014110207 location_1 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/ZdX9jTN7GCI= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014110207 location_1 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/EsD36WzmhzE= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014110207 location_3 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/p/nv5UT0hkk= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014110207 location_3 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/TnofqvefdXI= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014110207 location_4 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/5qWdSQrhHV0= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014110207 location_1 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/8r5r46hPAEw= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014110207 location_4 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/VIVmL0MGWSU= AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014110208 location_1 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/wlS8e8NE9Fo= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014110208 location_4 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/T+8eEM7m1Uw= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014110208 location_2 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/Z8guCudNNxg= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014110208 location_3 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/U0rR4dshSnE= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014110208 location_5 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/5xVssgtVKls= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014110208 location_4 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/h8iyvyQj4ww= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014110208 location_4 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/XkOS+Z2zjkA= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014110208 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/ED1vP/zC5Wo= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014110208 location_5 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/9GkMUQVpJgU= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014110208 location_3 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/yokbaX88njY= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014110209 location_3 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/vK2cf4JbyFU= AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014110209 location_2 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/L1h+ZEcYHRg= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014110209 location_2 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/3JRx0eiOkVU= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014110209 location_3 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/saPM57mJZhQ= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014110209 location_5 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/d8EKCKmMLVw= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014110209 location_2 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/w6AsAnuAvgk= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014110209 location_3 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/ltvwGOzB1HU= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014110209 location_3 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/Vsb2diytZzs= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014110209 location_4 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/Ho58LPw66Hs= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014110209 location_5 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/LLls3jIWc0Y= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014110210 location_2 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/FPPcmsEC5F8= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014110210 location_4 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/OkLVBTWc2TI= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014110210 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/knPYdh2CDFw= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014110210 location_3 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/GIltrs4UXBY= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014110210 location_4 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/tAiFB9uTT2o= AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014110210 location_3 product_10 AwEDAAAAAgABAAAAAAAAAP////////9//+nqr0tu5Xk= AwEDAAAAAgABAAAAAAAAAP////////9/Je73qhRRHjw= +2014110210 location_4 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/rxx3mzSdfCU= AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014110210 location_3 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/Da1T2BnOaEM= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014110210 location_2 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/E9GvRTxPc0k= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014110210 location_3 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/Y00PaHFpyWw= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014110211 location_3 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/HBz/TZPkphQ= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014110211 location_4 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/nPKHhcUCAzw= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014110211 location_5 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/I16DIaW90XI= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014110211 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/lWub/FYW8BQ= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014110211 location_2 product_7 AwEDAAAAAgABAAAAAAAAAP////////9//80ZjybDKzo= AwEDAAAAAgABAAAAAAAAAP////////9/C7HkJPudcg4= +2014110211 location_1 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/A44nockpekc= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014110211 location_5 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/WJYmme7F9TI= AwEDAAAAAgABAAAAAAAAAP////////9/af6UlZ8mGiQ= +2014110211 location_4 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/mP9/SeQlvxU= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014110211 location_1 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/Qc1yG8QMtgI= AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014110211 location_4 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/Fn7sJEOy4E0= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014110212 location_4 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/aFI4KGHLaUY= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014110212 location_1 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/gt2hN2gAsj0= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014110212 location_1 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/KEZCmnk/Enw= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014110212 location_5 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/S1XML6WIhV0= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014110212 location_2 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/8YQ3JucVtyU= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014110212 location_2 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/UjVB7P5peXw= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014110212 location_1 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/nxMvemioeSw= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014110212 location_5 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/BBffeFz9Rgg= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014110212 location_1 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/nYLwC41v+XI= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014110212 location_3 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/XoT7iHJ3Gns= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014110213 location_2 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/Qa1GJJFhbyA= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014110213 location_5 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/QQzXgcb3TlA= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014110213 location_3 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/CiH8NZB+9D4= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014110213 location_5 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/4m3dbIeB/Rw= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014110213 location_2 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/U37A0Cak5Ag= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014110213 location_1 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/CZuuOtN8WC4= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014110213 location_3 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/oadfngWebgk= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014110213 location_4 product_4 AwEDAAAAAgABAAAAAAAAAP////////9//WtMEZIKaG8= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014110213 location_4 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/CpoqQfgIklE= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014110213 location_5 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/iSTepHt8tDA= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014110214 location_5 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/9oj4iyH12Dw= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014110214 location_1 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/hVquUFv73Xs= AwEDAAAAAgABAAAAAAAAAP////////9/bvSh3SRDc3k= +2014110214 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/1RHnGA+TzQ8= AwEDAAAAAgABAAAAAAAAAP////////9/wossxL0+HGg= +2014110214 location_4 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/N7rDIP5WrDI= AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014110214 location_5 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/4NXKdcEMp0I= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014110214 location_3 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/uP6gPaGWMjc= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014110214 location_3 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/opx0NPEHrG8= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014110214 location_2 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/hC2GGuPvITw= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014110214 location_4 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/vi9i89oHA0s= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014110214 location_2 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/2cXIcuaULVk= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014110215 location_1 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/XaDOMRIP82g= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014110215 location_5 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/fNf6YQo+oEI= AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014110215 location_1 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/M7TENuosSQ8= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014110215 location_1 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/Dg71cBo5wFE= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014110215 location_4 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/FOHPObSYdz0= AwEDAAAAAgABAAAAAAAAAP////////9/tR9dunpTLSQ= +2014110215 location_3 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/0fN+LPxFo1Q= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014110215 location_2 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/3s8jlZtVLk4= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014110215 location_4 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/Xy1Vml/ypV0= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014110215 location_5 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/OLUpmjmHqCQ= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014110215 location_5 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/Vq/8inhFHHk= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014110216 location_5 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/vdOmtzjjJxE= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014110216 location_2 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/poafNgduXWw= AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014110216 location_2 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/sObbCr9mylU= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014110216 location_1 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/aRVwxTqno14= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014110216 location_2 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/ff4SwRioHl4= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014110216 location_3 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/wFhm6MZqWgY= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014110216 location_3 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/Xk6usrJoikg= AwEDAAAAAgABAAAAAAAAAP////////9/J+zV+RnYt2M= +2014110216 location_5 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/Cuix/DlO12I= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014110216 location_4 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/v3vOtPi8SmE= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014110216 location_5 product_4 AwEDAAAAAgABAAAAAAAAAP////////9/rv7BHQWwWyE= AwEDAAAAAgABAAAAAAAAAP////////9/H8YZlbHE03g= +2014110217 location_3 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/8rZ6gJ+ztmk= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014110217 location_1 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/BjkoRvD6d2E= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014110217 location_2 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/3FMQkqQ4RmE= AwEDAAAAAgABAAAAAAAAAP////////9/4EyBvXLM/xs= +2014110217 location_4 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/wXW2SNTHniU= AwEDAAAAAgABAAAAAAAAAP////////9/cJx3p0WZekM= +2014110217 location_2 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/kYiPnE1DCw8= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014110217 location_3 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/aA4MS1+p4RI= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014110217 location_3 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/9hTiMqPyEEM= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014110217 location_1 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/NC+Z/0JWoB8= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014110217 location_5 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/dKqF7WMTuH4= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014110217 location_1 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/LPrBhx8VkgQ= AwEDAAAAAgABAAAAAAAAAP////////9/PzNis25KfDo= +2014110218 location_2 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/OGwMjhqF0hE= AwEDAAAAAgABAAAAAAAAAP////////9/BnXke/a7yDI= +2014110218 location_2 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/dhqDNJnYzkg= AwEDAAAAAgABAAAAAAAAAP////////9/SlCxAVB4E1o= +2014110218 location_1 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/d2U+t+TrTQA= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014110218 location_2 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/OxUFxWpPdCo= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014110218 location_1 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/qnq2fx44URo= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014110218 location_1 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/+sAQDix4tUo= AwEDAAAAAgABAAAAAAAAAP////////9/ke63h/cmGSc= +2014110218 location_5 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/Cu569Vs9IH0= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014110218 location_5 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/edHLOvd49nQ= AwEDAAAAAgABAAAAAAAAAP////////9/L05RVtSVmSI= +2014110218 location_5 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/0J+WQlqNe28= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014110218 location_2 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/USS/ybhfyjE= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014110219 location_2 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/JlIzlHEQ2wk= AwEDAAAAAgABAAAAAAAAAP////////9/FF5eeCgr3W8= +2014110219 location_3 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/KQU9RJ9bQX4= AwEDAAAAAgABAAAAAAAAAP////////9/pF5fWpcjl1Q= +2014110219 location_1 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/735qVCCJxyw= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014110219 location_1 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/lC9Dskr+AS0= AwEDAAAAAgABAAAAAAAAAP////////9/NJVTgFqIyGg= +2014110219 location_4 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/3Eefo4TWx2E= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014110219 location_3 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/76EoHFeb+Q8= AwEDAAAAAgABAAAAAAAAAP////////9/L/RaCEd0lEo= +2014110219 location_3 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/pQxP1PISWT8= AwEDAAAAAgABAAAAAAAAAP////////9/RM/PPPc6GH4= +2014110219 location_2 product_11 AwEDAAAAAgABAAAAAAAAAP////////9//Qw9uappBUg= AwEDAAAAAgABAAAAAAAAAP////////9/wOBWrCfrXVs= +2014110219 location_4 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/LrBg1ogUMmE= AwEDAAAAAgABAAAAAAAAAP////////9/i3zstpLhWWs= +2014110219 location_5 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/lQqUp076bjA= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014110220 location_3 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/HXYbxYjrI2w= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014110220 location_3 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/rVV5eWpBej4= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014110220 location_3 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/PHQJ7WLdXBE= AwEDAAAAAgABAAAAAAAAAP////////9/q7QpTMZYNns= +2014110220 location_4 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/ttPQPGsknio= AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014110220 location_3 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/MtgccINinWc= AwEDAAAAAgABAAAAAAAAAP////////9/RI5olqYUtnQ= +2014110220 location_4 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/0AhnuYJKW2Y= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014110220 location_5 product_10 AwEDAAAAAgABAAAAAAAAAP////////9/Kl15sBKGvwg= AwEDAAAAAgABAAAAAAAAAP////////9/kV36ypNb+hM= +2014110220 location_5 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/81Crw4k0Z1Q= AwEDAAAAAgABAAAAAAAAAP////////9/fp5jsENLzEs= +2014110220 location_3 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/U+T8T9FL5jU= AwEDAAAAAgABAAAAAAAAAP////////9/Emg4u9g/BVM= +2014110220 location_3 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/NdCXrARHMQc= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014110221 location_4 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/1cn1C1rovmM= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014110221 location_4 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/yEW9PJYjx1k= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014110221 location_4 product_8 AwEDAAAAAgABAAAAAAAAAP////////9/MA2BbW3p1WE= AwEDAAAAAgABAAAAAAAAAP////////9/GedjOSzb7zs= +2014110221 location_4 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/rv7BHQWwWyE= AwEDAAAAAgABAAAAAAAAAP////////9/9ehUclgDjnA= +2014110221 location_3 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/doRdr3CBth0= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014110221 location_2 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/K/VnrIL6wGM= AwEDAAAAAgABAAAAAAAAAP////////9/DTn8nRFFr38= +2014110221 location_3 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/3DX+7ru55mY= AwEDAAAAAgABAAAAAAAAAP////////9/h3XDlGJinVg= +2014110221 location_4 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/VyffUoJiVS8= AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014110221 location_1 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/jxjTaiIn9nc= AwEDAAAAAgABAAAAAAAAAP////////9/ac/6/cikcwQ= +2014110221 location_1 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/T+PVJ49WZB8= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014110222 location_4 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/HAbhk/rDDTM= AwEDAAAAAgABAAAAAAAAAP////////9//BA41YVjDFI= +2014110222 location_2 product_11 AwEDAAAAAgABAAAAAAAAAP////////9/5PH9/yT0g00= AwEDAAAAAgABAAAAAAAAAP////////9/j+e0cICJPFg= +2014110222 location_4 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/h1bAreYUv0U= AwEDAAAAAgABAAAAAAAAAP////////9/jDwgknTL/S0= +2014110222 location_1 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/EYD/m8cNCz4= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= +2014110222 location_5 product_9 AwEDAAAAAgABAAAAAAAAAP////////9/I8Y+umpLF18= AwEDAAAAAgABAAAAAAAAAP////////9/dJ1rHBj48G0= +2014110222 location_2 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/3Vd3VU28Djw= AwEDAAAAAgABAAAAAAAAAP////////9/srmtgZz8Slg= +2014110222 location_3 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/yW8HrQ/3VC0= AwEDAAAAAgABAAAAAAAAAP////////9/Rn5CRuhP3h4= +2014110222 location_1 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/9mVpa8Jp9T0= AwEDAAAAAgABAAAAAAAAAP////////9/Czv0sOyV7j4= +2014110222 location_2 product_14 AwEDAAAAAgABAAAAAAAAAP////////9/ZanI2mnZ5gg= AwEDAAAAAgABAAAAAAAAAP////////9/q08lHAK+X3A= +2014110222 location_5 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/a0lNxRGeGyk= AwEDAAAAAgABAAAAAAAAAP////////9/PZr5ma3+TRQ= +2014110223 location_2 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/bEGc2mwSyyg= AwEDAAAAAgABAAAAAAAAAP////////9/l9m5EDihmD0= +2014110223 location_3 product_5 AwEDAAAAAgABAAAAAAAAAP////////9/C4LToyHPCww= AwEDAAAAAgABAAAAAAAAAP////////9/c7D7fxngbxU= +2014110223 location_2 product_7 AwEDAAAAAgABAAAAAAAAAP////////9/y4ezL/k/GA0= AwEDAAAAAgABAAAAAAAAAP////////9/KKcfz0hRe38= +2014110223 location_4 product_2 AwEDAAAAAgABAAAAAAAAAP////////9/l4NSpYnPLgM= AwEDAAAAAgABAAAAAAAAAP////////9/VqUKjaiiSUg= +2014110223 location_2 product_3 AwEDAAAAAgABAAAAAAAAAP////////9/qWW+CRi0uRI= AwEDAAAAAgABAAAAAAAAAP////////9/EOmCmVerjm4= +2014110223 location_5 product_1 AwEDAAAAAgABAAAAAAAAAP////////9/EtNnyx66Dmk= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014110223 location_1 product_15 AwEDAAAAAgABAAAAAAAAAP////////9/0AxfWShkkx4= AwEDAAAAAgABAAAAAAAAAP////////9/o31ldGC0E2s= +2014110223 location_5 product_6 AwEDAAAAAgABAAAAAAAAAP////////9/Wi1iJmzKZlM= AwEDAAAAAgABAAAAAAAAAP////////9/TqyltB+ASmU= +2014110223 location_1 product_13 AwEDAAAAAgABAAAAAAAAAP////////9/A5LtYs2Z3mE= AwEDAAAAAgABAAAAAAAAAP////////9/V3s8Tt0yshQ= +2014110223 location_2 product_12 AwEDAAAAAgABAAAAAAAAAP////////9/IlGEu4Rv9XQ= AwEDAAAAAgABAAAAAAAAAP////////9//mseeN0UsgU= diff --git a/integration-tests-ex/cases/resources/data/batch_index/json/wikipedia_index_data1.json b/integration-tests-ex/cases/resources/data/batch_index/json/wikipedia_index_data1.json new file mode 100644 index 00000000000..169796cd746 --- /dev/null +++ b/integration-tests-ex/cases/resources/data/batch_index/json/wikipedia_index_data1.json @@ -0,0 +1,3 @@ +{"timestamp": "2013-08-31T01:02:33Z", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 57, "deleted": 200, "delta": -143} +{"timestamp": "2013-08-31T03:32:45Z", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 459, "deleted": 129, "delta": 330} +{"timestamp": "2013-08-31T07:11:21Z", "page": "Cherno Alpha", "language" : "ru", "user" : "masterYi", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"article", "continent":"Asia", "country":"Russia", "region":"Oblast", "city":"Moscow", "added": 123, "deleted": 12, "delta": 111} \ No newline at end of file diff --git a/integration-tests-ex/cases/resources/data/batch_index/json/wikipedia_index_data2.json b/integration-tests-ex/cases/resources/data/batch_index/json/wikipedia_index_data2.json new file mode 100644 index 00000000000..62e270113d5 --- /dev/null +++ b/integration-tests-ex/cases/resources/data/batch_index/json/wikipedia_index_data2.json @@ -0,0 +1,3 @@ +{"timestamp": "2013-08-31T11:58:39Z", "page": "Crimson Typhoon", "language" : "zh", "user" : "triplets", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"China", "region":"Shanxi", "city":"Taiyuan", "added": 905, "deleted": 5, "delta": 900} +{"timestamp": "2013-08-31T12:41:27Z", "page": "Coyote Tango", "language" : "ja", "user" : "stringer", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"Japan", "region":"Kanto", "city":"Tokyo", "added": 1, "deleted": 10, "delta": -9} +{"timestamp": "2013-09-01T01:02:33Z", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 57, "deleted": 200, "delta": -143} \ No newline at end of file diff --git a/integration-tests-ex/cases/resources/data/batch_index/json/wikipedia_index_data3.json b/integration-tests-ex/cases/resources/data/batch_index/json/wikipedia_index_data3.json new file mode 100644 index 00000000000..28e0762f84c --- /dev/null +++ b/integration-tests-ex/cases/resources/data/batch_index/json/wikipedia_index_data3.json @@ -0,0 +1,4 @@ +{"timestamp": "2013-09-01T03:32:45Z", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 459, "deleted": 129, "delta": 330} +{"timestamp": "2013-09-01T07:11:21Z", "page": "Cherno Alpha", "language" : "ru", "user" : "masterYi", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"article", "continent":"Asia", "country":"Russia", "region":"Oblast", "city":"Moscow", "added": 123, "deleted": 12, "delta": 111} +{"timestamp": "2013-09-01T11:58:39Z", "page": "Crimson Typhoon", "language" : "zh", "user" : "triplets", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"China", "region":"Shanxi", "city":"Taiyuan", "added": 905, "deleted": 5, "delta": 900} +{"timestamp": "2013-09-01T12:41:27Z", "page": "Coyote Tango", "language" : "ja", "user" : "stringer", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"Japan", "region":"Kanto", "city":"Tokyo", "added": 1, "deleted": 10, "delta": -9} \ No newline at end of file diff --git a/integration-tests-ex/cases/resources/data/batch_index/orc/wikipedia_index_data1.orc b/integration-tests-ex/cases/resources/data/batch_index/orc/wikipedia_index_data1.orc new file mode 100644 index 00000000000..897b090e3b9 Binary files /dev/null and b/integration-tests-ex/cases/resources/data/batch_index/orc/wikipedia_index_data1.orc differ diff --git a/integration-tests-ex/cases/resources/data/batch_index/orc/wikipedia_index_data2.orc b/integration-tests-ex/cases/resources/data/batch_index/orc/wikipedia_index_data2.orc new file mode 100644 index 00000000000..3f0927403c0 Binary files /dev/null and b/integration-tests-ex/cases/resources/data/batch_index/orc/wikipedia_index_data2.orc differ diff --git a/integration-tests-ex/cases/resources/data/batch_index/orc/wikipedia_index_data3.orc b/integration-tests-ex/cases/resources/data/batch_index/orc/wikipedia_index_data3.orc new file mode 100644 index 00000000000..5c5f54486a0 Binary files /dev/null and b/integration-tests-ex/cases/resources/data/batch_index/orc/wikipedia_index_data3.orc differ diff --git a/integration-tests-ex/cases/resources/data/batch_index/parquet/wikipedia_index_data1.parquet b/integration-tests-ex/cases/resources/data/batch_index/parquet/wikipedia_index_data1.parquet new file mode 100644 index 00000000000..aaed7672c5d Binary files /dev/null and b/integration-tests-ex/cases/resources/data/batch_index/parquet/wikipedia_index_data1.parquet differ diff --git a/integration-tests-ex/cases/resources/data/batch_index/parquet/wikipedia_index_data2.parquet b/integration-tests-ex/cases/resources/data/batch_index/parquet/wikipedia_index_data2.parquet new file mode 100644 index 00000000000..6f26f1f07e3 Binary files /dev/null and b/integration-tests-ex/cases/resources/data/batch_index/parquet/wikipedia_index_data2.parquet differ diff --git a/integration-tests-ex/cases/resources/data/batch_index/parquet/wikipedia_index_data3.parquet b/integration-tests-ex/cases/resources/data/batch_index/parquet/wikipedia_index_data3.parquet new file mode 100644 index 00000000000..fd318da28b8 Binary files /dev/null and b/integration-tests-ex/cases/resources/data/batch_index/parquet/wikipedia_index_data3.parquet differ diff --git a/integration-tests-ex/cases/resources/data/batch_index/tsv/wikipedia_index_data1.tsv b/integration-tests-ex/cases/resources/data/batch_index/tsv/wikipedia_index_data1.tsv new file mode 100644 index 00000000000..b13d9adca1f --- /dev/null +++ b/integration-tests-ex/cases/resources/data/batch_index/tsv/wikipedia_index_data1.tsv @@ -0,0 +1,4 @@ +timestamp page language user unpatrolled newPage robot anonymous namespace continent country region city added deleted delta +2013-08-31T01:02:33Z Gypsy Danger en nuclear true true false false article North America United States Bay Area San Francisco 57 200 -143 +2013-08-31T03:32:45Z Striker Eureka en speed false true true false wikipedia Australia Australia Cantebury Syndey 459 129 330 +2013-08-31T07:11:21Z Cherno Alpha ru masterYi false true true false article Asia Russia Oblast Moscow 123 12 111 diff --git a/integration-tests-ex/cases/resources/data/batch_index/tsv/wikipedia_index_data2.tsv b/integration-tests-ex/cases/resources/data/batch_index/tsv/wikipedia_index_data2.tsv new file mode 100644 index 00000000000..e271291911c --- /dev/null +++ b/integration-tests-ex/cases/resources/data/batch_index/tsv/wikipedia_index_data2.tsv @@ -0,0 +1,4 @@ +timestamp page language user unpatrolled newPage robot anonymous namespace continent country region city added deleted delta +2013-08-31T11:58:39Z Crimson Typhoon zh triplets true false true false wikipedia Asia China Shanxi Taiyuan 905 5 900 +2013-08-31T12:41:27Z Coyote Tango ja stringer true false true false wikipedia Asia Japan Kanto Tokyo 1 10 -9 +2013-09-01T01:02:33Z Gypsy Danger en nuclear true true false false article North America United States Bay Area San Francisco 57 200 -143 \ No newline at end of file diff --git a/integration-tests-ex/cases/resources/data/batch_index/tsv/wikipedia_index_data3.tsv b/integration-tests-ex/cases/resources/data/batch_index/tsv/wikipedia_index_data3.tsv new file mode 100644 index 00000000000..b91eae4fde8 --- /dev/null +++ b/integration-tests-ex/cases/resources/data/batch_index/tsv/wikipedia_index_data3.tsv @@ -0,0 +1,5 @@ +timestamp page language user unpatrolled newPage robot anonymous namespace continent country region city added deleted delta +2013-09-01T03:32:45Z Striker Eureka en speed false true true false wikipedia Australia Australia Cantebury Syndey 459 129 330 +2013-09-01T07:11:21Z Cherno Alpha ru masterYi false true true false article Asia Russia Oblast Moscow 123 12 111 +2013-09-01T11:58:39Z Crimson Typhoon zh triplets true false true false wikipedia Asia China Shanxi Taiyuan 905 5 900 +2013-09-01T12:41:27Z Coyote Tango ja stringer true false true false wikipedia Asia Japan Kanto Tokyo 1 10 -9 \ No newline at end of file diff --git a/integration-tests-ex/cases/resources/indexer/wikipedia_combining_index_data.json b/integration-tests-ex/cases/resources/indexer/wikipedia_combining_index_data.json new file mode 100644 index 00000000000..fb7ca6ef70d --- /dev/null +++ b/integration-tests-ex/cases/resources/indexer/wikipedia_combining_index_data.json @@ -0,0 +1,3 @@ +{"timestamp": "2013-08-31T05:18:22Z", "page": "Gypsy Danger", "language" : "zh", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 977, "deleted": 200, "delta": -143} +{"timestamp": "2013-08-31T17:57:01Z", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 459, "deleted": 129, "delta": 330} +{"timestamp": "2013-09-01T18:22:39Z", "page": "Cherno Alpha", "language" : "ru", "user" : "masterYo", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"article", "continent":"Asia", "country":"Russia", "region":"Oblast", "city":"Moscow", "added": 123, "deleted": 12, "delta": 111} \ No newline at end of file diff --git a/integration-tests-ex/cases/resources/indexer/wikipedia_index_queries_only_data3.json b/integration-tests-ex/cases/resources/indexer/wikipedia_index_queries_only_data3.json new file mode 100644 index 00000000000..5838c247d74 --- /dev/null +++ b/integration-tests-ex/cases/resources/indexer/wikipedia_index_queries_only_data3.json @@ -0,0 +1,198 @@ +[ + { + "description": "timeseries, 1 agg, all", + "query":{ + "queryType" : "timeBoundary", + "dataSource": "%%DATASOURCE%%" + }, + "expectedResults":[ + { + "timestamp" : "2013-09-01T03:32:45.000Z", + "result" : { + "minTime" : "2013-09-01T03:32:45.000Z", + "maxTime" : "2013-09-01T12:41:27.000Z" + } + } + ] + }, + { + "description": "timeseries, datasketch aggs, all", + "query":{ + "queryType" : "timeseries", + "dataSource": "%%DATASOURCE%%", + "granularity":"day", + "intervals":[ + "2013-08-31T00:00/2013-09-01T00:00" + ], + "filter":null, + "aggregations":[ + { + "type": "HLLSketchMerge", + "name": "approxCountHLL", + "fieldName": "HLLSketchBuild", + "lgK": 12, + "tgtHllType": "HLL_4", + "round": true + }, + { + "type":"thetaSketch", + "name":"approxCountTheta", + "fieldName":"thetaSketch", + "size":16384, + "shouldFinalize":true, + "isInputThetaSketch":false, + "errorBoundsStdDev":null + }, + { + "type":"quantilesDoublesSketch", + "name":"quantilesSketch", + "fieldName":"quantilesDoublesSketch", + "k":128 + } + ] + }, + "expectedResults":[] + }, + { + "description": "timeseries, datasketch aggs, all", + "query":{ + "queryType" : "timeseries", + "dataSource": "%%DATASOURCE%%", + "granularity":"day", + "intervals":[ + "2013-09-01T00:00/2013-09-02T00:00" + ], + "filter":null, + "aggregations":[ + { + "type": "HLLSketchMerge", + "name": "approxCountHLL", + "fieldName": "HLLSketchBuild", + "lgK": 12, + "tgtHllType": "HLL_4", + "round": true + }, + { + "type":"thetaSketch", + "name":"approxCountTheta", + "fieldName":"thetaSketch", + "size":16384, + "shouldFinalize":true, + "isInputThetaSketch":false, + "errorBoundsStdDev":null + }, + { + "type":"quantilesDoublesSketch", + "name":"quantilesSketch", + "fieldName":"quantilesDoublesSketch", + "k":128 + } + ] + }, + "expectedResults":[ + { + "timestamp" : "2013-09-01T00:00:00.000Z", + "result" : { + "quantilesSketch":4, + "approxCountTheta":4.0, + "approxCountHLL":4 + } + } + ] + }, + { + "description":"having spec on post aggregation", + "query":{ + "queryType":"groupBy", + "dataSource":"%%DATASOURCE%%", + "granularity":"day", + "dimensions":[ + "page" + ], + "filter":{ + "type":"selector", + "dimension":"language", + "value":"zh" + }, + "aggregations":[ + { + "type":"count", + "name":"rows" + }, + { + "type":"longSum", + "fieldName":"added", + "name":"added_count" + } + ], + "postAggregations": [ + { + "type":"arithmetic", + "name":"added_count_times_ten", + "fn":"*", + "fields":[ + {"type":"fieldAccess", "name":"added_count", "fieldName":"added_count"}, + {"type":"constant", "name":"const", "value":10} + ] + } + ], + "having":{"type":"greaterThan", "aggregation":"added_count_times_ten", "value":9000}, + "intervals":[ + "2013-08-31T00:00/2013-09-01T00:00" + ] + }, + "expectedResults":[] + }, + { + "description":"having spec on post aggregation", + "query":{ + "queryType":"groupBy", + "dataSource":"%%DATASOURCE%%", + "granularity":"day", + "dimensions":[ + "page" + ], + "filter":{ + "type":"selector", + "dimension":"language", + "value":"zh" + }, + "aggregations":[ + { + "type":"count", + "name":"rows" + }, + { + "type":"longSum", + "fieldName":"added", + "name":"added_count" + } + ], + "postAggregations": [ + { + "type":"arithmetic", + "name":"added_count_times_ten", + "fn":"*", + "fields":[ + {"type":"fieldAccess", "name":"added_count", "fieldName":"added_count"}, + {"type":"constant", "name":"const", "value":10} + ] + } + ], + "having":{"type":"greaterThan", "aggregation":"added_count_times_ten", "value":9000}, + "intervals":[ + "2013-09-01T00:00/2013-09-02T00:00" + ] + }, + "expectedResults":[ { + "version" : "v1", + "timestamp" : "2013-09-01T00:00:00.000Z", + "event" : { + "added_count_times_ten" : 9050.0, + "page" : "Crimson Typhoon", + "added_count" : 905, + "rows" : 1 + } + } ] + } +] \ No newline at end of file diff --git a/integration-tests-ex/cases/resources/indexer/wikipedia_realtime_index_data.json b/integration-tests-ex/cases/resources/indexer/wikipedia_realtime_index_data.json new file mode 100644 index 00000000000..626bb2df861 --- /dev/null +++ b/integration-tests-ex/cases/resources/indexer/wikipedia_realtime_index_data.json @@ -0,0 +1,22 @@ +{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 1, "deleted": 1, "delta": 0}, +{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 3, "deleted": 3, "delta": 0}, +{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Cherno Alpha", "language" : "ru", "user" : "masterYi", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"article", "continent":"Asia", "country":"Russia", "region":"Oblast", "city":"Moscow", "added": 5, "deleted": 5, "delta": 0}, +{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 7, "deleted": 3, "delta": 4} +{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Coyote Tango", "language" : "ja", "user" : "stringer", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"Japan", "region":"Kanto", "city":"Tokyo", "added": 11, "deleted": 11, "delta": 0}, +{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 13, "deleted": 13, "delta": 0}, +{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 17, "deleted": 17, "delta": 0}, +{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 19, "deleted": 19, "delta": 0}, +{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Cherno Alpha", "language" : "ru", "user" : "masterYi", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"article", "continent":"Asia", "country":"Russia", "region":"Oblast", "city":"Moscow", "added": 23, "deleted": 23, "delta": 0}, +{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 29, "deleted": 31, "delta": -1} +{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Coyote Tango", "language" : "ja", "user" : "stringer", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"Japan", "region":"Kanto", "city":"Tokyo", "added": 31, "deleted": 31, "delta": 0}, +{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 37, "deleted": 37, "delta": 0}, +{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 41, "deleted": 41, "delta": 0}, +{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 43, "deleted": 43, "delta": 0}, +{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Cherno Alpha", "language" : "ru", "user" : "masterYi", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"article", "continent":"Asia", "country":"Russia", "region":"Oblast", "city":"Moscow", "added": 47, "deleted": 47, "delta": 0}, +{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Crimson Typhoon", "language" : "zh", "user" : "triplets", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"China", "region":"Shanxi", "city":"Taiyuan", "added": 905, "deleted": 5, "delta": 900} +{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Coyote Tango", "language" : "ja", "user" : "stringer", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"Japan", "region":"Kanto", "city":"Tokyo", "added": 59, "deleted": 59, "delta": 0}, +{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 61, "deleted": 61, "delta": 0}, +{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 67, "deleted": 67, "delta": 0}, +{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 71, "deleted": 71, "delta": 0}, +{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 73, "deleted": 73, "delta": 0}, +{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Cherno Alpha", "language" : "ru", "user" : "masterYi", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"article", "continent":"Asia", "country":"Russia", "region":"Oblast", "city":"Moscow", "added": 79, "deleted": 79, "delta": 0} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/categories/BatchIndex.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/categories/BatchIndex.java new file mode 100644 index 00000000000..f11bfb6f60f --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/categories/BatchIndex.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.categories; + +public class BatchIndex +{ + +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/categories/HighAvailability.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/categories/HighAvailability.java new file mode 100644 index 00000000000..b12ba7f23e1 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/categories/HighAvailability.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.categories; + +public class HighAvailability +{ + +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/cluster/DruidClusterClient.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/cluster/DruidClusterClient.java new file mode 100644 index 00000000000..88ee6c0afd7 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/cluster/DruidClusterClient.java @@ -0,0 +1,340 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.cluster; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.RE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.jackson.JacksonUtils; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.java.util.http.client.HttpClient; +import org.apache.druid.java.util.http.client.Request; +import org.apache.druid.java.util.http.client.response.StatusResponseHandler; +import org.apache.druid.java.util.http.client.response.StatusResponseHolder; +import org.apache.druid.server.DruidNode; +import org.apache.druid.testing.guice.TestClient; +import org.apache.druid.testsEx.config.ResolvedConfig; +import org.apache.druid.testsEx.config.ResolvedDruidService; +import org.apache.druid.testsEx.config.ResolvedService.ResolvedInstance; +import org.jboss.netty.handler.codec.http.HttpMethod; +import org.jboss.netty.handler.codec.http.HttpResponseStatus; + +import javax.inject.Inject; + +import java.io.IOException; +import java.net.URL; +import java.util.Map; + +/** + * Client to the Druid cluster described by the test cluster + * configuration. Various clients exist for specific services or tasks: + * this client is about the cluster as a whole, with operations used + * by tests. + */ +public class DruidClusterClient +{ + private static final Logger log = new Logger(DruidClusterClient.class); + + private final ResolvedConfig config; + private final HttpClient httpClient; + private final ObjectMapper jsonMapper; + + @Inject + public DruidClusterClient( + ResolvedConfig config, + @TestClient HttpClient httpClient, + ObjectMapper jsonMapper + ) + { + this.config = config; + this.httpClient = httpClient; + this.jsonMapper = jsonMapper; + } + + public ResolvedConfig config() + { + return config; + } + + /** + * Return the configuration object for the leader for + * the given service. + */ + public ResolvedInstance leader(ResolvedDruidService service) + { + if (service.requireInstances().size() == 1) { + return service.instance(); + } + String leader = getLeader(service.service()); + return service.findHost(leader); + } + + /** + * Returns the leader URL for the given service. + */ + public String getLeader(String service) + { + String url = StringUtils.format( + "%s/druid/%s/v1/leader", + config.routerUrl(), + service + ); + return get(url).getContent(); + } + + /** + * Checks if a node is healthy, given the service and instance. + * + * @return `true` if the message returns `true`, `false` if the + * message fails (indicating the node is not healthy.) + */ + public boolean isHealthy(ResolvedDruidService service, ResolvedInstance instance) + { + return isHealthy(service.resolveUrl(instance)); + } + + /** + * Checks if a node is healthy given the URL for that node. + * + * @return `true` if the message returns `true`, `false` if the + * message fails (indicating the node is not healthy.) + */ + public boolean isHealthy(String serviceUrl) + { + try { + String url = StringUtils.format( + "%s/status/health", + serviceUrl + ); + return getAs(url, Boolean.class); + } + catch (Exception e) { + return false; + } + } + + /** + * Returns the URL for the lead coordinator. + */ + public String leadCoordinatorUrl() + { + ResolvedDruidService coord = config.requireCoordinator(); + ResolvedInstance leader = leader(coord); + return coord.resolveUrl(leader); + } + + /** + * Calls the `/v1/cluster` endpoint on the lead coordinator. + */ + public Map coordinatorCluster() + { + String url = StringUtils.format( + "%s/druid/coordinator/v1/cluster", + leadCoordinatorUrl() + ); + return getAs(url, JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT); + } + + /** + * Calls the `/v1/cluster` endpoint on the router. + */ + public Map routerCluster() + { + String url = StringUtils.format( + "%s/druid/router/v1/cluster", + config.routerUrl() + ); + return getAs(url, JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT); + } + + /** + * Low-level HTTP get for the given URL. + */ + public StatusResponseHolder get(String url) + { + try { + StatusResponseHolder response = httpClient.go( + new Request(HttpMethod.GET, new URL(url)), + StatusResponseHandler.getInstance() + ).get(); + + if (!response.getStatus().equals(HttpResponseStatus.OK)) { + throw new ISE( + "Error from GET [%s] status [%s] content [%s]", + url, + response.getStatus(), + response.getContent() + ); + } + return response; + } + catch (Exception e) { + throw new RuntimeException(e); + } + } + + /** + * Issue a GET command and deserialize the JSON result to the given class. + */ + public T getAs(String url, Class clazz) + { + StatusResponseHolder response = get(url); + try { + return jsonMapper.readValue(response.getContent(), clazz); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + + /** + * Issue a GET command and deserialize the JSON result to the given type reference. + */ + public T getAs(String url, TypeReference typeRef) + { + StatusResponseHolder response = get(url); + try { + return jsonMapper.readValue(response.getContent(), typeRef); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + + /** + * Call the `/status/selfDiscovered` given a node URL. + */ + public boolean selfDiscovered(String nodeUrl) + { + String url = StringUtils.format( + "%s/status/selfDiscovered", + nodeUrl + ); + try { + get(url); + } + catch (Exception e) { + return false; + } + return true; + } + + /** + * Validates the cluster by waiting for each service declared in the + * test configuration to report that it is healthy. By doing this at the + * start of the test, individual tests don't have to retry to handle the + * race condition that otherwise occurs between cluster and test startup. + */ + public void validate() + { + log.info("Starting cluster validation"); + for (ResolvedDruidService service : config.requireDruid().values()) { + for (ResolvedInstance instance : service.requireInstances()) { + validateInstance(service, instance); + } + } + log.info("Cluster validated."); + } + + /** + * Validate an instance by waiting for it to report that it is healthy. + */ + private void validateInstance(ResolvedDruidService service, ResolvedInstance instance) + { + int timeoutMs = config.readyTimeoutSec() * 1000; + int pollMs = config.readyPollMs(); + long startTime = System.currentTimeMillis(); + long updateTime = startTime + 5000; + while (System.currentTimeMillis() - startTime < timeoutMs) { + if (isHealthy(service, instance)) { + log.info( + "Service %s, host %s is ready", + service.service(), + instance.clientHost()); + return; + } + long currentTime = System.currentTimeMillis(); + if (currentTime > updateTime) { + log.info( + "Service %s, host %s not ready, retrying", + service.service(), + instance.clientHost()); + updateTime = currentTime + 5000; + } + try { + Thread.sleep(pollMs); + } + catch (InterruptedException e) { + throw new RuntimeException("Interrupted during cluster validation"); + } + } + throw new RE( + StringUtils.format("Service %s, instance %s not ready after %d ms.", + service.service(), + instance.tag() == null ? "" : instance.tag(), + timeoutMs)); + } + + /** + * Wait for an instance to become ready given the URL and a description of + * the service. + */ + public void waitForNodeReady(String label, String url) + { + int timeoutMs = config.readyTimeoutSec() * 1000; + int pollMs = config.readyPollMs(); + long startTime = System.currentTimeMillis(); + while (System.currentTimeMillis() - startTime < timeoutMs) { + if (isHealthy(url)) { + log.info( + "Service %s, url %s is ready", + label, + url); + return; + } + log.info( + "Service %s, url %s not ready, retrying", + label, + url); + try { + Thread.sleep(pollMs); + } + catch (InterruptedException e) { + throw new RuntimeException("Interrupted while waiting for note to be ready"); + } + } + throw new RE( + StringUtils.format("Service %s, url %s not ready after %d ms.", + label, + url, + timeoutMs)); + } + + public String nodeUrl(DruidNode node) + { + return StringUtils.format( + "http://%s:%s", + config.hasProxy() ? config.proxyHost() : node.getHost(), + node.getPlaintextPort() + ); + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/cluster/KafkaClient.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/cluster/KafkaClient.java new file mode 100644 index 00000000000..91da889c685 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/cluster/KafkaClient.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.cluster; + +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.testsEx.config.ResolvedService.ResolvedKafka; +import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.clients.admin.DescribeClusterResult; + +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.ExecutionException; + +/** + * Simple Kafka client for a test cluster. + *

+ * At present, this client contains just enough functionality to validate + * that Kafka is up. It can be expanded to gather Kafka code currently + * copy/pasted in tests. + */ +public class KafkaClient +{ + private final ResolvedKafka config; + private Admin admin; + + public KafkaClient(ResolvedKafka config) + { + this.config = config; + } + + public void open() + { + validate(); + } + + public Admin adminClient() + { + if (admin == null) { + final Map props = new HashMap<>(); + props.put("bootstrap.servers", config.bootstrap()); + admin = Admin.create(props); + } + return admin; + } + + public void validate() + { + DescribeClusterResult result = adminClient().describeCluster(); + try { + if (result.nodes().get().isEmpty()) { + throw new ISE("No nodes found in Kafka cluster"); + } + } + catch (InterruptedException | ExecutionException e) { + throw new ISE(e, "Could not connect to Kafka"); + } + } + + public void close() + { + if (admin != null) { + admin.close(); + admin = null; + } + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/cluster/MetastoreClient.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/cluster/MetastoreClient.java new file mode 100644 index 00000000000..86e0ece8d52 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/cluster/MetastoreClient.java @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.cluster; + +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.metadata.MetadataStorageConnector; +import org.apache.druid.metadata.SQLMetadataConnector; +import org.skife.jdbi.v2.DBI; +import org.skife.jdbi.v2.Handle; + +import javax.inject.Inject; + +import java.sql.Connection; +import java.sql.SQLException; + +/** + * Simple test-time client to the MySQL metastore. + *

+ * Used to verify that the DB is up and available. The JDBC + * connection can be used to query the metadata DB for tests. + *

+ * Also handles running metastore setup queries on test + * startup. Write such queries to be idempotent: REPLACE + * rather than INSERT, for example. + */ +public class MetastoreClient +{ + // See SQLMetadataConnector.getValidationQuery() + // That instance isn't available here, so we punt. + public static String VALIDATION_QUERY = "SELECT 1"; + + private DBI dbi; + private Handle handle; + + @Inject + public MetastoreClient(MetadataStorageConnector connector) + { + SQLMetadataConnector sqlConnector = (SQLMetadataConnector) connector; + this.dbi = sqlConnector.getDBI(); + this.handle = dbi.open(); + } + + public void validate() + { + boolean ok = execute(VALIDATION_QUERY); + if (!ok) { + throw new ISE("Metadata store validation failed"); + } + } + + public Connection connection() + { + return handle.getConnection(); + } + + public boolean execute(String sql) + { + try { + return connection().prepareStatement(sql).execute(); + } + catch (SQLException e) { + throw new ISE(e, "Metadata query failed"); + } + } + + public void close() + { + handle.close(); + handle = null; + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/cluster/ZooKeeperClient.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/cluster/ZooKeeperClient.java new file mode 100644 index 00000000000..cad00d1ae4a --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/cluster/ZooKeeperClient.java @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.cluster; + +import org.apache.curator.ensemble.EnsembleProvider; +import org.apache.curator.framework.CuratorFramework; +import org.apache.druid.curator.CuratorConfig; +import org.apache.druid.curator.CuratorModule; +import org.apache.druid.curator.ExhibitorConfig; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.testsEx.config.ResolvedConfig; +import org.apache.druid.testsEx.config.ResolvedService.ResolvedZk; + +import java.util.concurrent.TimeUnit; + +/** + * Test oriented ZooKeeper client. + *

+ * Currently contains just enough functionality to verify that + * ZK is ready. + */ +public class ZooKeeperClient +{ + private final ResolvedConfig clusterConfig; + private final ResolvedZk config; + private CuratorFramework curatorFramework; + + public ZooKeeperClient(ResolvedConfig config) + { + this.clusterConfig = config; + this.config = config.zk(); + if (this.config == null) { + throw new ISE("ZooKeeper not configured"); + } + prepare(); + awaitReady(); + } + + private void prepare() + { + CuratorConfig curatorConfig = clusterConfig.toCuratorConfig(); + ExhibitorConfig exhibitorConfig = clusterConfig.toExhibitorConfig(); + EnsembleProvider ensembleProvider = CuratorModule.createEnsembleProvider(curatorConfig, exhibitorConfig); + curatorFramework = CuratorModule.createCurator(curatorConfig, ensembleProvider); + } + + private void awaitReady() + { + int timeoutSec = config.startTimeoutSecs(); + if (timeoutSec == 0) { + timeoutSec = 5; + } + try { + curatorFramework.start(); + curatorFramework.blockUntilConnected(timeoutSec, TimeUnit.SECONDS); + } + catch (InterruptedException e) { + throw new ISE("ZooKeeper timed out waiting for connect"); + } + } + + public CuratorFramework curator() + { + return curatorFramework; + } + + public void close() + { + curatorFramework.close(); + curatorFramework = null; + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/Cluster.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/Cluster.java new file mode 100644 index 00000000000..d45ec864d2e --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/Cluster.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.config; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Inherited; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + * Indicates the cluster configuration to use for a test. If omitted, the + * cluster configuration is the same as the category. Use this annotation + * on a category class to indicate that category shares configuration with + * another category. Example:

{@code
+ * @Cluster(BatchIndex.class)
+ * public class InputFormat
+ * {
+ * }
+ * Note that a category uses a single cluster configuration. It is not + * possible to have tests choose the cluster type because Maven must + * start the cluster before running tests, thus clusters are a category-level + * concept, not a test-level concept. + */ +@Retention(RetentionPolicy.RUNTIME) +@Inherited +@Target(ElementType.TYPE) +public @interface Cluster +{ + Class value(); +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ClusterConfig.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ClusterConfig.java new file mode 100644 index 00000000000..363f7648b07 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ClusterConfig.java @@ -0,0 +1,342 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.config; + +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonInclude.Include; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.dataformat.yaml.YAMLFactory; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.testsEx.config.ServiceConfig.DruidConfig; +import org.apache.druid.testsEx.config.ServiceConfig.ZKConfig; + +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Java representation of the test configuration YAML. + *

+ * This object is primarily de-serialized as the files are written by hand + * to define a test. It is serialized only for debugging. + */ +public class ClusterConfig +{ + public enum ClusterType + { + docker, + k8s, + local, + disabled + } + + private boolean isResource; + private String basePath; + + @JsonProperty("type") + private ClusterType type; + @JsonProperty("proxyHost") + private String proxyHost; + @JsonProperty("include") + private List include; + @JsonProperty("readyTimeoutSec") + private int readyTimeoutSec; + @JsonProperty("readyPollMs") + private int readyPollMs; + @JsonProperty("zk") + private ZKConfig zk; + @JsonProperty("metastore") + private MetastoreConfig metastore; + @JsonProperty("kafka") + private KafkaConfig kafka; + @JsonProperty("druid") + private Map druidServices; + @JsonProperty("settings") + private Map settings; + @JsonProperty("properties") + private Map properties; + @JsonProperty("metastoreInit") + private List metastoreInit; + @JsonProperty("datasourceSuffix") + private String datasourceSuffix; + + /** + * Delay after initializing the DB to wait for the coordinator to notice + * the changes. This is a pure hack to work around an optimization in + * the coordinator that would otherwise ignore the changes. + * Set to a bit longer than the coordinator druid.manager.segments.pollDuration + * property. + */ + @JsonProperty("metastoreInitDelaySec") + private int metastoreInitDelaySec; + + public ClusterConfig() + { + } + + public ClusterConfig(ClusterConfig from) + { + this.type = from.type; + this.proxyHost = from.proxyHost; + this.include = null; // Tell IntelliJ inspections we don't want to copy this. + this.readyTimeoutSec = from.readyTimeoutSec; + this.readyPollMs = from.readyPollMs; + this.isResource = from.isResource; + this.basePath = from.basePath; + this.metastoreInitDelaySec = from.metastoreInitDelaySec; + this.datasourceSuffix = from.datasourceSuffix; + if (from.include != null) { + this.include = new ArrayList<>(from.include); + } + this.zk = from.zk; + this.metastore = from.metastore; + this.kafka = from.kafka; + if (from.druidServices != null) { + this.druidServices = new HashMap<>(from.druidServices); + } + if (from.properties != null) { + this.properties = new HashMap<>(from.properties); + } + if (from.settings != null) { + this.settings = new HashMap<>(from.settings); + } + if (from.metastoreInit != null) { + this.metastoreInit = new ArrayList<>(from.metastoreInit); + } + } + + public static ClusterConfig loadFromFile(String filePath) + { + return loadFromFile(new File(filePath)); + } + + public static ClusterConfig loadFromFile(File configFile) + { + ObjectMapper mapper = new ObjectMapper(new YAMLFactory()); + try { + ClusterConfig config = mapper.readValue(configFile, ClusterConfig.class); + config.isResource = false; + config.basePath = configFile.getParent(); + return config; + } + catch (IOException e) { + throw new ISE(e, "Failed to load config file: " + configFile); + } + } + + public static ClusterConfig loadFromResource(String resource) + { + ObjectMapper mapper = new ObjectMapper(new YAMLFactory()); + try (InputStream is = TestConfigs.class.getResourceAsStream(resource)) { + if (is == null) { + throw new ISE("Config resource not found: " + resource); + } + ClusterConfig config = mapper.readValue(is, ClusterConfig.class); + config.isResource = true; + return config; + } + catch (IOException e) { + throw new ISE(e, "Failed to load config resource: " + resource); + } + } + + public ResolvedConfig resolve(String clusterName) + { + return new ResolvedConfig(clusterName, resolveIncludes()); + } + + public ClusterConfig resolveIncludes() + { + if (include == null || include.isEmpty()) { + return this; + } + ClusterConfig included = null; + for (String entry : include) { + ClusterConfig child = loadInclude(entry); + if (included == null) { + included = child; + } else { + included = included.merge(child); + } + } + return included.merge(this); + } + + private ClusterConfig loadInclude(String includeName) + { + if (isResource) { + return loadFromResource(includeName); + } else { + File file = new File(new File(basePath), includeName); + return loadFromFile(file); + } + } + + @JsonProperty("type") + @JsonInclude(Include.NON_DEFAULT) + public ClusterType type() + { + return type; + } + + @JsonProperty("readyTimeoutSec") + @JsonInclude(Include.NON_DEFAULT) + public int readyTimeoutSec() + { + return readyTimeoutSec; + } + + @JsonProperty("readyPollMs") + @JsonInclude(Include.NON_DEFAULT) + public int readyPollMs() + { + return readyPollMs; + } + + @JsonProperty("proxyHost") + @JsonInclude(Include.NON_NULL) + public String proxyHost() + { + return proxyHost; + } + + @JsonProperty("include") + @JsonInclude(Include.NON_NULL) + public List include() + { + return include; + } + + @JsonProperty("zk") + @JsonInclude(Include.NON_NULL) + public ZKConfig zk() + { + return zk; + } + + @JsonProperty("metastore") + @JsonInclude(Include.NON_NULL) + public MetastoreConfig metastore() + { + return metastore; + } + + @JsonProperty("kafka") + @JsonInclude(Include.NON_NULL) + public KafkaConfig kafka() + { + return kafka; + } + + @JsonProperty("druid") + @JsonInclude(Include.NON_NULL) + public Map druid() + { + return druidServices; + } + + @JsonProperty("settings") + @JsonInclude(Include.NON_NULL) + public Map settings() + { + return settings; + } + + @JsonProperty("properties") + @JsonInclude(Include.NON_NULL) + public Map properties() + { + return properties; + } + + @JsonProperty("metastoreInit") + @JsonInclude(Include.NON_NULL) + public List metastoreInit() + { + return metastoreInit; + } + + @JsonProperty("metastoreInitDelaySec") + @JsonInclude(Include.NON_DEFAULT) + public int metastoreInitDelaySec() + { + return metastoreInitDelaySec; + } + + @JsonProperty("datasourceSuffix") + @JsonInclude(Include.NON_NULL) + public String datasourceSuffix() + { + return datasourceSuffix; + } + + @Override + public String toString() + { + return TestConfigs.toYaml(this); + } + + public ClusterConfig merge(ClusterConfig overrides) + { + ClusterConfig merged = new ClusterConfig(this); + if (overrides.readyTimeoutSec != 0) { + merged.readyTimeoutSec = overrides.readyTimeoutSec; + } + if (overrides.proxyHost != null) { + merged.proxyHost = overrides.proxyHost; + } + // Includes are already considered. + if (overrides.zk != null) { + merged.zk = overrides.zk; + } + if (overrides.metastore != null) { + merged.metastore = overrides.metastore; + } + if (overrides.kafka != null) { + merged.kafka = overrides.kafka; + } + if (merged.druidServices == null) { + merged.druidServices = overrides.druidServices; + } else if (overrides.druidServices != null) { + merged.druidServices.putAll(overrides.druidServices); + } + if (merged.settings == null) { + merged.settings = overrides.settings; + } else if (overrides.settings != null) { + merged.settings.putAll(overrides.settings); + } + if (merged.properties == null) { + merged.properties = overrides.properties; + } else if (overrides.properties != null) { + merged.properties.putAll(overrides.properties); + } + if (merged.metastoreInit == null) { + merged.metastoreInit = overrides.metastoreInit; + } else if (overrides.metastoreInit != null) { + merged.metastoreInit.addAll(overrides.metastoreInit); + } + return merged; + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ClusterConfigTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ClusterConfigTest.java new file mode 100644 index 00000000000..1531edff0fd --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ClusterConfigTest.java @@ -0,0 +1,123 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.config; + +import org.apache.druid.testing.IntegrationTestingConfig; +import org.apache.druid.testsEx.config.ClusterConfig.ClusterType; +import org.apache.druid.testsEx.config.ResolvedService.ResolvedZk; +import org.junit.Test; + +import java.io.File; +import java.io.FileNotFoundException; +import java.io.FileOutputStream; +import java.io.OutputStreamWriter; +import java.io.PrintWriter; +import java.nio.charset.StandardCharsets; +import java.util.Map; +import java.util.Properties; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; + +/** + * Sanity check of an example YAML config file using the Java + * deserialization classes. + */ +public class ClusterConfigTest +{ + @Test + public void testYaml() throws FileNotFoundException + { + ClusterConfig config = ClusterConfig.loadFromResource("/config-test/test.yaml"); + // Uncomment this line to see the full config with includes resolved. + //System.out.println(config.resolveIncludes()); + + ResolvedConfig resolved = config.resolve("Test"); + assertEquals(ClusterType.docker, resolved.type()); + assertEquals(ResolvedConfig.DEFAULT_READY_TIMEOUT_SEC, resolved.readyTimeoutSec()); + assertEquals(ResolvedConfig.DEFAULT_READY_POLL_MS, resolved.readyPollMs()); + assertEquals(3, resolved.properties().size()); + + ResolvedZk zk = resolved.zk(); + assertNotNull(zk); + assertEquals("zookeeper", zk.service()); + assertEquals(1, zk.requireInstances().size()); + assertEquals(2181, zk.instance().port()); + assertEquals(2181, zk.instance().clientPort()); + assertEquals("zookeeper", zk.instance().host()); + assertEquals("localhost", zk.instance().clientHost()); + assertEquals("zookeeper:2181", zk.clusterHosts()); + assertEquals("localhost:2181", zk.clientHosts()); + + ResolvedMetastore ms = resolved.metastore(); + assertNotNull(ms); + assertEquals("metastore", ms.service()); + assertEquals(1, ms.requireInstances().size()); + assertEquals("jdbc:mysql://localhost:3306/druid", ms.connectURI()); + assertEquals("druid", ms.user()); + assertEquals("diurd", ms.password()); + + ResolvedDruidService service = resolved.requireBroker(); + assertNotNull(service); + assertEquals("broker", service.service()); + assertEquals("http://localhost:8082", service.clientUrl()); + + service = resolved.requireRouter(); + assertNotNull(service); + assertEquals("router", service.service()); + assertEquals("http://localhost:8888", service.clientUrl()); + assertEquals("http://localhost:8888", resolved.routerUrl()); + + File userEnv = new File( + new File( + System.getProperty("user.home"), + "druid-it"), + "Test.env"); + try (PrintWriter out = new PrintWriter(new OutputStreamWriter(new FileOutputStream(userEnv), StandardCharsets.UTF_8))) { + out.println("druid_user_var=user"); + } + + System.setProperty("druid_sys_prop", "sys"); + Map props = resolved.toProperties(); + // Added from ZK section + assertEquals("localhost:2181", props.get("druid.zk.service.zkHosts")); + // Generic property + assertEquals("howdy", props.get("my.test.property")); + // Mapped from settings + assertEquals("myBucket", props.get("druid.test.config.cloudBucket")); + assertEquals("myPath", props.get("druid.test.config.cloudPath")); + assertEquals("secret", props.get("druid.test.config.s3AccessKey")); + // From settings, overridden in properties + assertEquals("myRegion", props.get("druid.test.config.cloudRegion")); + // System property + assertEquals("sys", props.get("druid.test.config.sys_prop")); + // From user override + assertEquals("user", props.get("druid.test.config.user_var")); + + // Test plumbing through the test config + Properties properties = new Properties(); + properties.putAll(props); + IntegrationTestingConfig testingConfig = new IntegrationTestingConfigEx(resolved, properties); + assertEquals("myBucket", testingConfig.getCloudBucket()); + assertEquals("myPath", testingConfig.getCloudPath()); + // From settings, overridden in properties + assertEquals("myRegion", testingConfig.getCloudRegion()); + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/Configure.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/Configure.java new file mode 100644 index 00000000000..99e8b7e3cbb --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/Configure.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.config; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + * Indicates a method to use to configure the {@link Initializer.Builder}. Must be of the + * form: + *


+ * @Configure
+ * public static void configure(Initializer.Builder builder)
+ * {
+ *    builder.something(arg);
+ * }
+ * 
+ */
+@Retention(RetentionPolicy.RUNTIME)
+@Target(ElementType.METHOD)
+public @interface Configure
+{
+}
diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/DruidTestRunner.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/DruidTestRunner.java
new file mode 100644
index 00000000000..5e65ec7f9d4
--- /dev/null
+++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/DruidTestRunner.java
@@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.testsEx.config;
+
+import junitparams.JUnitParamsRunner;
+import org.apache.druid.java.util.common.UOE;
+import org.junit.experimental.categories.Category;
+import org.junit.runners.model.InitializationError;
+import org.junit.runners.model.Statement;
+
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+
+/**
+ * Test runner for Druid integration tests. Loads test configuration. Initializes
+ * Guice. Injects dependencies into the test. Runs the tests.
+ * Shuts down afterwards. Since cluster configuration and health checks are a bit
+ * expensive, does that work once per test class rather than once per test method.
+ * 

+ * Note that our Guice usage is a bit awkward for tests. Druid modules define + * objects that must be lifecycle managed, but as lazy singletons, which means + * that they might be created after the lifecycle starts, which causes the dreaded + * "it doesn't work that way" message. The awkward workaround is to ask to inject + * test members before starting the lifecycle, so that the injection creates + * a reference, which creates the object, which registers it in the lifecycle. We + * should fix this issue. Until then, the awkwardness is hidden in this test runner. + *

+ * Extends the parameterize test runner, so your Druid ITs can also use parameters. + */ +public class DruidTestRunner extends JUnitParamsRunner +{ + private class CloseInitializer extends Statement + { + private final Statement next; + + public CloseInitializer(Statement next) + { + this.next = next; + } + + @Override + public void evaluate() throws Throwable + { + next.evaluate(); + if (initializer != null) { + initializer.close(); + initializer = null; + } + } + } + + private Initializer initializer; + + public DruidTestRunner(Class testClass) throws InitializationError + { + super(testClass); + } + + @Override + protected Object createTest() throws Exception + { + Object test = super.createTest(); + if (initializer == null) { + initializer = buildInitializer(test); + } else { + initializer.injector().injectMembers(test); + } + return test; + } + + private Initializer buildInitializer(Object test) + { + Class testClass = test.getClass(); + Category[] annotations = testClass.getAnnotationsByType(Category.class); + if (annotations.length == 0) { + throw new UOE( + "Class % must have a @Category annotation", + testClass.getSimpleName() + ); + } + if (annotations.length != 1) { + throw new UOE( + "Class % must have exactly one @Category annotation", + testClass.getSimpleName() + ); + } + Class[] categories = annotations[0].value(); + if (categories.length == 0) { + throw new UOE( + "Class % must have a @Category value", + testClass.getSimpleName() + ); + } + if (categories.length != 1) { + throw new UOE( + "Class % must have exactly one @Category value", + testClass.getSimpleName() + ); + } + Class category = category(testClass); + String clusterName = inferCluster(category); + Initializer.Builder builder = Initializer.builder(clusterName) + .test(test) + .validateCluster(); + for (Method method : testClass.getMethods()) { + if (method.getAnnotation(Configure.class) == null) { + continue; + } + final int requiredMods = Modifier.STATIC | Modifier.PUBLIC; + if ((method.getModifiers() & requiredMods) != requiredMods) { + throw new UOE( + "Method %s annotated with @Configure must be public static", + method.getName() + ); + } + try { + method.invoke(null, builder); + } + catch (IllegalAccessException | IllegalArgumentException | InvocationTargetException e) { + throw new UOE( + "Call to Method %s annotated with @Configure failed: %s", + method.getName(), + e.getMessage() + ); + } + } + return builder.build(); + } + + /** + * Resolve the {@code @Category} annotation for the test class. + */ + private Class category(Class testClass) + { + Category[] annotations = testClass.getAnnotationsByType(Category.class); + if (annotations.length == 0) { + throw new UOE( + "Class % must have a @Category annotation", + testClass.getSimpleName() + ); + } + if (annotations.length != 1) { + throw new UOE( + "Class % must have exactly one @Category annotation", + testClass.getSimpleName() + ); + } + Class[] categories = annotations[0].value(); + if (categories.length == 0) { + throw new UOE( + "Class % must have a @Category value", + testClass.getSimpleName() + ); + } + if (categories.length != 1) { + throw new UOE( + "Class % must have exactly one @Category value", + testClass.getSimpleName() + ); + } + return categories[0]; + } + + /** + * Resolve the optional {@code @Cluster} annotation on the test category. + * If omitted, the category itself is the cluster name. + */ + private String inferCluster(Class category) + { + String categoryName = category.getSimpleName(); + Cluster[] annotations = category.getAnnotationsByType(Cluster.class); + if (annotations.length == 0) { + return categoryName; + } + if (annotations.length != 1) { + throw new UOE( + "Category % must have no more than one @Cluster annotation", + category.getSimpleName() + ); + } + Class clusterClass = annotations[0].value(); + return clusterClass.getSimpleName(); + } + + @Override + protected Statement withAfterClasses(Statement statement) + { + return new CloseInitializer(super.withAfterClasses(statement)); + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/Initializer.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/Initializer.java new file mode 100644 index 00000000000..a2899a08448 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/Initializer.java @@ -0,0 +1,590 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.config; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.module.SimpleModule; +import com.google.common.collect.ImmutableList; +import com.google.inject.Binder; +import com.google.inject.Injector; +import com.google.inject.Key; +import com.google.inject.Module; +import com.google.inject.Provides; +import org.apache.druid.cli.GuiceRunnable; +import org.apache.druid.curator.CuratorModule; +import org.apache.druid.curator.discovery.DiscoveryModule; +import org.apache.druid.discovery.DruidNodeDiscoveryProvider; +import org.apache.druid.guice.AnnouncerModule; +import org.apache.druid.guice.DruidProcessingConfigModule; +import org.apache.druid.guice.JsonConfigProvider; +import org.apache.druid.guice.LazySingleton; +import org.apache.druid.guice.ManageLifecycle; +import org.apache.druid.guice.PolyBind; +import org.apache.druid.guice.SQLMetadataStorageDruidModule; +import org.apache.druid.guice.StartupInjectorBuilder; +import org.apache.druid.guice.StorageNodeModule; +import org.apache.druid.guice.annotations.Client; +import org.apache.druid.guice.annotations.EscalatedClient; +import org.apache.druid.guice.annotations.Self; +import org.apache.druid.guice.http.HttpClientModule; +import org.apache.druid.guice.security.EscalatorModule; +import org.apache.druid.initialization.CoreInjectorBuilder; +import org.apache.druid.initialization.DruidModule; +import org.apache.druid.jackson.DruidServiceSerializerModifier; +import org.apache.druid.jackson.StringObjectPairList; +import org.apache.druid.jackson.ToStringObjectPairListDeserializer; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.lifecycle.Lifecycle; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.java.util.emitter.core.LoggingEmitter; +import org.apache.druid.java.util.emitter.core.LoggingEmitterConfig; +import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.java.util.http.client.HttpClient; +import org.apache.druid.metadata.MetadataStorageConnector; +import org.apache.druid.metadata.MetadataStorageConnectorConfig; +import org.apache.druid.metadata.MetadataStorageProvider; +import org.apache.druid.metadata.MetadataStorageTablesConfig; +import org.apache.druid.metadata.NoopMetadataStorageProvider; +import org.apache.druid.metadata.SQLMetadataConnector; +import org.apache.druid.metadata.storage.mysql.MySQLConnector; +import org.apache.druid.metadata.storage.mysql.MySQLConnectorDriverConfig; +import org.apache.druid.metadata.storage.mysql.MySQLConnectorSslConfig; +import org.apache.druid.metadata.storage.mysql.MySQLMetadataStorageModule; +import org.apache.druid.server.DruidNode; +import org.apache.druid.testing.IntegrationTestingConfig; +import org.apache.druid.testing.IntegrationTestingConfigProvider; +import org.apache.druid.testing.guice.TestClient; +import org.apache.druid.testsEx.cluster.DruidClusterClient; +import org.apache.druid.testsEx.cluster.MetastoreClient; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Properties; + +/** + * The magic needed to piece together enough of Druid to allow clients to + * run without server dependencies being pulled in. Used to set up the + * Guice injector used to inject members into integration tests, while + * reading configuration from the docker.yaml or similar test + * configuration file. + *

+ * Much of the work here deals the tedious task of assembling Druid + * modules, sometimes using copy/past to grab the part that a client + * wants (such as object deserialization) without the parts that the + * server needs (and which would introduce the need for unused configuration + * just to make dependencies work.) + *

+ * See the documentation for these test for the "user view" of this + * class and its configuration. + */ +public class Initializer +{ + public static final String TEST_CONFIG_PROPERTY = "testConfig"; + public static final String TEST_CONFIG_VAR = "TEST_CONFIG"; + public static final String CLUSTER_RESOURCES = "/cluster/"; + public static final String CLUSTER_CONFIG_RESOURCE = CLUSTER_RESOURCES + "%s/%s.yaml"; + public static final String CLUSTER_CONFIG_DEFAULT = "docker"; + public static final String METASTORE_CONFIG_PROPERTY = "sqlConfig"; + public static final String METASTORE_CONFIG_RESOURCE = "/metastore/%s.sql"; + public static final String METASTORE_CONFIG_DEFAULT = "init"; + + private static final Logger log = new Logger(Initializer.class); + + public static String queryFile(Class category, String fileName) + { + return CLUSTER_RESOURCES + category.getSimpleName() + "/queries/" + fileName; + } + + private static class TestModule implements DruidModule + { + ResolvedConfig config; + + public TestModule(ResolvedConfig config) + { + this.config = config; + } + + @Override + public void configure(Binder binder) + { + binder + .bind(ResolvedConfig.class) + .toInstance(config); + binder + .bind(IntegrationTestingConfig.class) + .to(IntegrationTestingConfigEx.class) + .in(LazySingleton.class); + binder + .bind(MetastoreClient.class) + .in(LazySingleton.class); + + // Dummy DruidNode instance to make Guice happy. This instance is unused. + binder + .bind(DruidNode.class) + .annotatedWith(Self.class) + .toInstance( + new DruidNode("integration-tests", "localhost", false, 9191, null, null, true, false)); + + // Reduced form of SQLMetadataStorageDruidModule + String prop = SQLMetadataStorageDruidModule.PROPERTY; + String defaultValue = MySQLMetadataStorageModule.TYPE; + PolyBind.createChoiceWithDefault(binder, prop, Key.get(MetadataStorageConnector.class), defaultValue); + PolyBind.createChoiceWithDefault(binder, prop, Key.get(MetadataStorageProvider.class), defaultValue); + PolyBind.createChoiceWithDefault(binder, prop, Key.get(SQLMetadataConnector.class), defaultValue); + + // Reduced form of MetadataConfigModule + // Not actually used here (tests don't create tables), but needed by MySQLConnector constructor + JsonConfigProvider.bind(binder, MetadataStorageTablesConfig.PROPERTY_BASE, MetadataStorageTablesConfig.class); + + // Build from properties provided in the config + JsonConfigProvider.bind(binder, MetadataStorageConnectorConfig.PROPERTY_BASE, MetadataStorageConnectorConfig.class); + } + + @Provides + @TestClient + public HttpClient getHttpClient( + IntegrationTestingConfig config, + Lifecycle lifecycle, + @Client HttpClient delegate + ) + { + return delegate; + } + + @Provides + @ManageLifecycle + public ServiceEmitter getServiceEmitter(ObjectMapper jsonMapper) + { + return new ServiceEmitter("", "", new LoggingEmitter(new LoggingEmitterConfig(), jsonMapper)); + } + + // From ServerModule to allow deserialization of DiscoveryDruidNode objects from ZK. + // We don't want the other dependencies of that module. + @Override + public List getJacksonModules() + { + return ImmutableList.of( + new SimpleModule() + .addDeserializer(StringObjectPairList.class, new ToStringObjectPairListDeserializer()) + .setSerializerModifier(new DruidServiceSerializerModifier()) + ); + } + } + + /** + * Reduced form of MySQLMetadataStorageModule. + */ + private static class TestMySqlModule implements DruidModule + { + @Override + public void configure(Binder binder) + { + JsonConfigProvider.bind(binder, "druid.metadata.mysql.ssl", MySQLConnectorSslConfig.class); + JsonConfigProvider.bind(binder, "druid.metadata.mysql.driver", MySQLConnectorDriverConfig.class); + String type = MySQLMetadataStorageModule.TYPE; + PolyBind + .optionBinder(binder, Key.get(MetadataStorageProvider.class)) + .addBinding(type) + .to(NoopMetadataStorageProvider.class) + .in(LazySingleton.class); + + PolyBind + .optionBinder(binder, Key.get(MetadataStorageConnector.class)) + .addBinding(type) + .to(MySQLConnector.class) + .in(LazySingleton.class); + + PolyBind + .optionBinder(binder, Key.get(SQLMetadataConnector.class)) + .addBinding(type) + .to(MySQLConnector.class) + .in(LazySingleton.class); + } + + @Override + public List getJacksonModules() + { + return new MySQLMetadataStorageModule().getJacksonModules(); + } + } + + /** + * Class used by test to identify test-specific options, load configuration + * and "inject themselves" with dependencies. + */ + public static class Builder + { + private final String clusterName; + private String configFile; + private Object test; + private List modules = new ArrayList<>(); + private boolean validateCluster; + private List> eagerCreation = new ArrayList<>(); + private Map envVarBindings = new HashMap<>(); + private Properties testProperties = new Properties(); + + public Builder(String clusterName) + { + this.clusterName = clusterName; + + // Node discovery is lifecycle managed. If we're using it, we have to + // create the instance within Guice during setup so it can be lifecycle + // managed. Using LazySingleon works in a server, but not in test clients, + // because test clients declare their need of node discovery after the + // the lifecycle starts. + eagerInstance(DruidNodeDiscoveryProvider.class); + + // Set properties from environment variables, or hard-coded values + // previously set in Maven. + propertyEnvVarBinding("druid.test.config.dockerIp", "DOCKER_IP"); + propertyEnvVarBinding("druid.zk.service.host", "DOCKER_IP"); + propertyEnvVarBinding("druid.test.config.hadoopDir", "HADOOP_DIR"); + property("druid.client.https.trustStorePath", "client_tls/truststore.jks"); + property("druid.client.https.trustStorePassword", "druid123"); + property("druid.client.https.keyStorePath", "client_tls/client.jks"); + property("druid.client.https.certAlias", "druid"); + property("druid.client.https.keyManagerPassword", "druid123"); + property("druid.client.https.keyStorePassword", "druid123"); + + // More env var bindings for properties formerly passed in via + // a generated config file. + final String base = IntegrationTestingConfigProvider.PROPERTY_BASE + "."; + propertyEnvVarBinding(base + "cloudBucket", "DRUID_CLOUD_BUCKET"); + propertyEnvVarBinding(base + "cloudPath", "DRUID_CLOUD_PATH"); + propertyEnvVarBinding(base + "s3AccessKey", "AWS_ACCESS_KEY_ID"); + propertyEnvVarBinding(base + "s3SecretKey", "AWS_SECRET_ACCESS_KEY"); + propertyEnvVarBinding(base + "azureContainer", "AZURE_CONTAINER"); + propertyEnvVarBinding(base + "azureAccount", "AZURE_ACCOUNT"); + propertyEnvVarBinding(base + "azureKey", "AZURE_KEY"); + propertyEnvVarBinding(base + "googleBucket", "GOOGLE_BUCKET"); + propertyEnvVarBinding(base + "googlePrefix", "GOOGLE_PREFIX"); + + // Other defaults + // druid.global.http.numMaxThreads avoids creating 40+ Netty threads. + // We only ever use 1. + property("druid.global.http.numMaxThreads", 3); + property("druid.broker.http.numMaxThreads", 3); + } + + /** + * Load a configuration from the named file. Primarily for + * debugging to use a one-off, custom configuration file. + */ + public Builder configFile(String configFile) + { + this.configFile = configFile; + return this; + } + + /** + * The test class with members to be injected. + */ + public Builder test(Object test) + { + this.test = test; + return this; + } + + /** + * Druid provides the {@code PolyBind} abstraction and the {@code Lifecycle} + * abstraction. When used together, we can encounter initialization issues. We won't create + * and instance of a polymorphic binding until it is first needed, and only then does + * the instance add itself to the lifecycle. However, if it is a test that acks for + * the instance, that is too late: the lifecycle has started. A test should call this + * method to "register" polymorphic lifecycle classes that will be injected later. + *

+ * The builder registers {@code DruidNodeDiscoveryProvider} by default: add any + * test-specific instances as needed. + */ + public Builder eagerInstance(Class theClass) + { + this.eagerCreation.add(theClass); + return this; + } + + /** + * Optional test-specific modules to load. + */ + public Builder modules(List modules) + { + this.modules.addAll(modules); + return this; + } + + public Builder modules(Module...modules) + { + return modules(Arrays.asList(modules)); + } + + /** + * Validates the cluster before running tests. Ensures that each + * Druid service reports itself as healthy. Since Druid services + * depend on ZK and the metadata DB, this indirectly checks their + * health as well. + */ + public Builder validateCluster() + { + this.validateCluster = true; + return this; + } + + /** + * Set a property value in code. Such values go into the {@link Properties} + * object in Guice, and act as defaults to properties defined in the config + * file or via system properties. These properties can also "hard code" items + * that would normally be user-settable in a server. The value can be of any + * type: it is converted to a String internally. + */ + public Builder property(String key, Object value) + { + if (value == null) { + testProperties.remove(key); + } else { + testProperties.put(key, value.toString()); + } + return this; + } + + /** + * Bind a property value to an environment variable. Useful if the property + * is set in the environment via the build system, Maven or other means. + * Avoids the need to add command-line arguments of the form + * {@code -Dproperty.name=$ENV_VAR}. Environment variable bindings take + * precedence over values set via {@link #property(String, Object)}, or + * the config file, but are lower priority than system properties. The + * environment variable is used only if set, else it is ignored. + */ + public Builder propertyEnvVarBinding(String property, String envVar) + { + this.envVarBindings.put(property, envVar); + return this; + } + + public synchronized Initializer build() + { + return new Initializer(this); + } + } + + private final ResolvedConfig clusterConfig; + private final Injector injector; + private final Lifecycle lifecycle; + private MetastoreClient metastoreClient; + private DruidClusterClient clusterClient; + + private Initializer(Builder builder) + { + if (builder.configFile != null) { + this.clusterConfig = loadConfigFile(builder.clusterName, builder.configFile); + } else { + this.clusterConfig = loadConfig(builder.clusterName, builder.configFile); + } + this.injector = makeInjector(builder, clusterConfig); + + // Do the injection of test members early, for force lazy singleton + // instance creation to avoid problems when lifecycle-managed objects + // are combined with PolyBind. + if (builder.test != null) { + this.injector.injectMembers(builder.test); + } + + // Backup: instantiate any additional instances that might be referenced + // later outside of injection. + for (Class eagerClass : builder.eagerCreation) { + this.injector.getInstance(eagerClass); + } + + // Now that we've created lifecycle-managed instances, start the lifecycle. + log.info("Starting lifecycle"); + this.lifecycle = GuiceRunnable.initLifecycle(injector, log); + + // Verify the cluster to ensure it is ready. + log.info("Creating cluster client"); + this.clusterClient = this.injector.getInstance(DruidClusterClient.class); + if (builder.validateCluster) { + clusterClient.validate(); + } + + // Now that the cluster is ready (which implies that the metastore is ready), + // load any "starter data" into the metastore. Warning: there is a time-lag between + // when the DB is updated and when Coordinator or Overlord learns about the updates. + // At present, there is no API to force a cache flush. Caveat emptor. + prepareDB(); + } + + public static Builder builder(String clusterName) + { + return new Builder(clusterName); + } + + private static ResolvedConfig loadConfig(String category, String configName) + { + if (configName == null) { + configName = System.getProperty(TEST_CONFIG_PROPERTY); + } + if (configName == null) { + configName = System.getenv(TEST_CONFIG_VAR); + } + if (configName == null) { + configName = CLUSTER_CONFIG_DEFAULT; + } + String loadName = StringUtils.format(CLUSTER_CONFIG_RESOURCE, category, configName); + ClusterConfig config = ClusterConfig.loadFromResource(loadName); + return config.resolve(category); + } + + private static ResolvedConfig loadConfigFile(String category, String path) + { + ClusterConfig config = ClusterConfig.loadFromFile(path); + return config.resolve(category); + } + + private static Injector makeInjector( + Builder builder, + ResolvedConfig clusterConfig + ) + { + Injector startupInjector = new StartupInjectorBuilder() + .withProperties(properties(builder, clusterConfig)) + .build(); + return new CoreInjectorBuilder(startupInjector) + .withLifecycle() + .add( + // Required by clients + new EscalatorModule(), + HttpClientModule.global(), + HttpClientModule.escalatedGlobal(), + new HttpClientModule("druid.broker.http", Client.class), + new HttpClientModule("druid.broker.http", EscalatedClient.class), + // For ZK discovery + new CuratorModule(), + new AnnouncerModule(), + new DiscoveryModule(), + // Dependencies from other modules + new DruidProcessingConfigModule(), + // Dependencies from other modules + new StorageNodeModule(), + + // Test-specific items, including bits copy/pasted + // from modules that don't play well in a client setting. + new TestModule(clusterConfig), + new TestMySqlModule() + ) + .addAll(builder.modules) + .build(); + } + + /** + * Define test properties similar to how the server does. Property precedence + * is: + *

+ */ + private static Properties properties( + Builder builder, + ResolvedConfig clusterConfig + ) + { + Properties finalProperties = new Properties(); + finalProperties.putAll(builder.testProperties); + finalProperties.putAll(clusterConfig.toProperties()); + for (Entry entry : builder.envVarBindings.entrySet()) { + String value = System.getenv(entry.getValue()); + if (value != null) { + finalProperties.put(entry.getKey(), value); + } + } + finalProperties.putAll(System.getProperties()); + log.info("Properties:"); + log.info(finalProperties.toString()); + return finalProperties; + } + + /** + * Some tests need a known set of metadata in the metadata DB. To avoid the + * complexity of do the actual actions (such as creating segments), the tests + * "seed" the database directly. The result is not entirely valid and consistent, + * but is good enough for the test at hand. + *

+ * WARNING: At present, there is no way to force the Coordinator or + * Overlord to flush its cache to learn about these new entries. Instead, we have + * to sleep for the cache timeout period. This solution is unsatisfying, and error-prone. + */ + private void prepareDB() + { + ResolvedMetastore metastoreConfig = clusterConfig.metastore(); + if (metastoreConfig == null) { + return; + } + List stmts = metastoreConfig.initStmts(); + if (stmts == null || stmts.isEmpty()) { + return; + } + log.info("Preparing database"); + MetastoreClient client = injector.getInstance(MetastoreClient.class); + for (MetastoreStmt stmt : stmts) { + client.execute(stmt.toSQL()); + } + try { + Thread.sleep(metastoreConfig.initDelaySec() * 1000); + } + catch (InterruptedException e) { + throw new RuntimeException("Interrupted while waiting for coordinator to notice DB changes"); + } + log.info("Database prepared"); + } + + public Injector injector() + { + return injector; + } + + public ResolvedConfig clusterConfig() + { + return clusterConfig; + } + + public MetastoreClient metastoreClient() + { + if (clusterConfig.metastore() == null) { + throw new IAE("Please provide a metastore section in docker.yaml"); + } + return injector.getInstance(MetastoreClient.class); + } + + public DruidClusterClient clusterClient() + { + return clusterClient; + } + + public void close() + { + lifecycle.stop(); + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/IntegrationTestingConfigEx.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/IntegrationTestingConfigEx.java new file mode 100644 index 00000000000..c14ea745aa8 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/IntegrationTestingConfigEx.java @@ -0,0 +1,421 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.config; + +import com.google.common.collect.ImmutableMap; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.testing.IntegrationTestingConfig; +import org.apache.druid.testing.IntegrationTestingConfigProvider; + +import javax.inject.Inject; + +import java.util.Map; +import java.util.Properties; + +/** + * Adapter to the "legacy" cluster configuration used by tests. + */ +class IntegrationTestingConfigEx implements IntegrationTestingConfig +{ + private final ResolvedConfig config; + private final Map properties; + + @Inject + public IntegrationTestingConfigEx( + final ResolvedConfig config, + final Properties properties) + { + this.config = config; + ImmutableMap.Builder builder = ImmutableMap.builder(); + for (Map.Entry entry : properties.entrySet()) { + String key = (String) entry.getKey(); + if (key.startsWith(IntegrationTestingConfigProvider.PROPERTY_BASE)) { + key = key.substring(IntegrationTestingConfigProvider.PROPERTY_BASE.length() + 1); + builder.put(key, (String) entry.getValue()); + } + } + this.properties = builder.build(); + } + + @Override + public String getZookeeperHosts() + { + return config.requireZk().clientHosts(); + } + + @Override + public String getKafkaHost() + { + return config.requireKafka().instance().clientHost(); + } + + @Override + public String getKafkaInternalHost() + { + return config.requireKafka().instance().host(); + } + + @Override + public String getBrokerHost() + { + return config.requireBroker().instance().clientHost(); + } + + @Override + public String getBrokerInternalHost() + { + return config.requireBroker().instance().host(); + } + + @Override + public String getRouterHost() + { + return config.requireRouter().instance().clientHost(); + } + + @Override + public String getRouterInternalHost() + { + return config.requireRouter().instance().host(); + } + + @Override + public String getCoordinatorHost() + { + return config.requireCoordinator().tagOrDefault("one").clientHost(); + } + + @Override + public String getCoordinatorInternalHost() + { + return config.requireCoordinator().tagOrDefault("one").host(); + } + + @Override + public String getCoordinatorTwoInternalHost() + { + return config.requireCoordinator().requireInstance("two").host(); + } + + @Override + public String getCoordinatorTwoHost() + { + return config.requireCoordinator().tagOrDefault("one").clientHost(); + } + + @Override + public String getOverlordHost() + { + return config.requireOverlord().tagOrDefault("one").clientHost(); + } + + @Override + public String getOverlordTwoHost() + { + return config.requireOverlord().tagOrDefault("two").clientHost(); + } + + @Override + public String getOverlordInternalHost() + { + return config.requireOverlord().tagOrDefault("one").host(); + } + + @Override + public String getOverlordTwoInternalHost() + { + return config.requireOverlord().requireInstance("two").host(); + } + + @Override + public String getMiddleManagerHost() + { + return config.requireMiddleManager().instance().clientHost(); + } + + @Override + public String getMiddleManagerInternalHost() + { + return config.requireMiddleManager().instance().host(); + } + + @Override + public String getHistoricalHost() + { + return config.requireHistorical().instance().clientHost(); + } + + @Override + public String getHistoricalInternalHost() + { + return config.requireHistorical().instance().host(); + } + + @Override + public String getCoordinatorUrl() + { + ResolvedDruidService serviceConfig = config.requireCoordinator(); + return serviceConfig.resolveUrl(serviceConfig.tagOrDefault("one")); + } + + @Override + public String getCoordinatorTLSUrl() + { + throw new ISE("Not implemented"); + } + + @Override + public String getCoordinatorTwoUrl() + { + ResolvedDruidService serviceConfig = config.requireCoordinator(); + return serviceConfig.resolveUrl(serviceConfig.requireInstance("two")); + } + + @Override + public String getCoordinatorTwoTLSUrl() + { + throw new ISE("Not implemented"); + } + + @Override + public String getOverlordUrl() + { + ResolvedDruidService serviceConfig = config.requireOverlord(); + return serviceConfig.resolveUrl(serviceConfig.tagOrDefault("one")); + } + + @Override + public String getOverlordTLSUrl() + { + throw new ISE("Not implemented"); + } + + @Override + public String getOverlordTwoUrl() + { + ResolvedDruidService serviceConfig = config.requireOverlord(); + return serviceConfig.resolveUrl(serviceConfig.requireInstance("two")); + } + + @Override + public String getOverlordTwoTLSUrl() + { + throw new ISE("Not implemented"); + } + + @Override + public String getIndexerUrl() + { + ResolvedDruidService indexer = config.druidService(ResolvedConfig.INDEXER); + if (indexer == null) { + indexer = config.requireMiddleManager(); + } + return indexer.resolveUrl(indexer.instance()); + } + + @Override + public String getIndexerTLSUrl() + { + throw new ISE("Not implemented"); + } + + @Override + public String getRouterUrl() + { + return config.routerUrl(); + } + + @Override + public String getRouterTLSUrl() + { + ResolvedDruidService serviceConfig = config.requireRouter(); + return serviceConfig.resolveUrl(serviceConfig.tagOrDefault("tls")); + } + + @Override + public String getPermissiveRouterUrl() + { + throw new ISE("Not implemented"); + } + + @Override + public String getPermissiveRouterTLSUrl() + { + throw new ISE("Not implemented"); + } + + @Override + public String getNoClientAuthRouterUrl() + { + throw new ISE("Not implemented"); + } + + @Override + public String getNoClientAuthRouterTLSUrl() + { + throw new ISE("Not implemented"); + } + + @Override + public String getCustomCertCheckRouterUrl() + { + throw new ISE("Not implemented"); + } + + @Override + public String getCustomCertCheckRouterTLSUrl() + { + throw new ISE("Not implemented"); + } + + @Override + public String getBrokerUrl() + { + ResolvedDruidService serviceConfig = config.requireBroker(); + return serviceConfig.resolveUrl(serviceConfig.instance()); + } + + @Override + public String getBrokerTLSUrl() + { + ResolvedDruidService serviceConfig = config.requireBroker(); + return serviceConfig.resolveUrl(serviceConfig.tagOrDefault("tls")); + } + + @Override + public String getHistoricalUrl() + { + return config.requireHistorical().resolveUrl(); + } + + @Override + public String getHistoricalTLSUrl() + { + throw new ISE("Not implemented"); + } + + @Override + public String getProperty(String prop) + { + return properties.get(prop); + } + + @Override + public String getUsername() + { + return getProperty("username"); + } + + @Override + public String getPassword() + { + return getProperty("password"); + } + + @Override + public Map getProperties() + { + return properties; + } + + @Override + public boolean manageKafkaTopic() + { + throw new ISE("Not implemented"); + } + + @Override + public String getExtraDatasourceNameSuffix() + { + return config.datasourceNameSuffix; + } + + @Override + public String getCloudBucket() + { + return getProperty("cloudBucket"); + } + + @Override + public String getCloudPath() + { + return getProperty("cloudPath"); + } + + @Override + public String getCloudRegion() + { + return getProperty("cloudRegion"); + } + + @Override + public String getS3AssumeRoleWithExternalId() + { + return getProperty("s3AssumeRoleWithExternalId"); + } + + @Override + public String getS3AssumeRoleExternalId() + { + return getProperty("s3AssumeRoleExternalId"); + } + + @Override + public String getS3AssumeRoleWithoutExternalId() + { + return getProperty("s3AssumeRoleWithoutExternalId"); + } + + @Override + public String getAzureKey() + { + return getProperty("azureKey"); + } + + @Override + public String getHadoopGcsCredentialsPath() + { + return getProperty("hadoopGcsCredentialsPath"); + } + + @Override + public String getStreamEndpoint() + { + return getProperty("streamEndpoint"); + } + + @Override + public String getSchemaRegistryHost() + { + return getProperty("schemaRegistryHost"); + } + + @Override + public boolean isDocker() + { + return config.isDocker(); + } + + @Override + public String getDockerHost() + { + return config.proxyHost(); + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/KafkaConfig.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/KafkaConfig.java new file mode 100644 index 00000000000..00e785d940d --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/KafkaConfig.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.config; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.List; + +public class KafkaConfig extends ServiceConfig +{ + @JsonCreator + public KafkaConfig( + @JsonProperty("service") String service, + @JsonProperty("instances") List instances + ) + { + super(service, instances); + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/MetastoreConfig.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/MetastoreConfig.java new file mode 100644 index 00000000000..cad2fd5293e --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/MetastoreConfig.java @@ -0,0 +1,121 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.config; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonInclude.Include; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Strings; + +import java.util.List; +import java.util.Map; + +public class MetastoreConfig extends ServiceConfig +{ + /** + * Driver. Defaults to the MySQL Driver. + * @see {@link org.apache.druid.metadata.storage.mysql.MySQLConnectorDriverConfig} + */ + private final String driver; + + /** + * JDBC connection URI. Required. + */ + private final String connectURI; + + /** + * User for the metastore DB. + */ + private final String user; + + /** + * Password for the metastore DB. + */ + private final String password; + + /** + * Optional connection properties. + */ + private final Map properties; + + @JsonCreator + public MetastoreConfig( + @JsonProperty("service") String service, + @JsonProperty("driver") String driver, + @JsonProperty("connectURI") String connectURI, + @JsonProperty("user") String user, + @JsonProperty("password") String password, + @JsonProperty("properties") Map properties, + @JsonProperty("instances") List instances + ) + { + super(service, instances); + this.driver = driver; + this.connectURI = connectURI; + this.user = user; + this.password = password; + this.properties = properties; + } + + @JsonProperty("driver") + @JsonInclude(Include.NON_NULL) + public String driver() + { + return driver; + } + + @JsonProperty("connectURI") + @JsonInclude(Include.NON_NULL) + public String connectURI() + { + return connectURI; + } + + @JsonProperty("user") + @JsonInclude(Include.NON_NULL) + public String user() + { + return user; + } + + @JsonProperty("password") + @JsonInclude(Include.NON_NULL) + public String password() + { + return password; + } + + @JsonProperty("properties") + @JsonInclude(Include.NON_NULL) + public Map properties() + { + return properties; + } + + public boolean validate(List errs) + { + if (Strings.isNullOrEmpty(connectURI)) { + errs.add("Metastore connect URI is required"); + return false; + } + return true; + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/MetastoreStmt.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/MetastoreStmt.java new file mode 100644 index 00000000000..5213678f562 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/MetastoreStmt.java @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.config; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.commons.lang3.RegExUtils; + +/** + * Represents a statement (query) to send to the Druid metadata + * storage database (metastore) before running tests. At present, + * each query is only a SQL statement. The statements are represented + * as objects to allow for other options (such as ignoring failures, + * etc.) + *

+ * Metastore queries often include a JSON payload. The metastore wants + * to store the payload in compact form without spaces. However, such + * JSON is hard for humans to understand. So, the configuration file + * should format the SQL statement and JSON for readability. This class + * will "compactify" the statement prior to execution. + */ +public class MetastoreStmt +{ + private final String sql; + + @JsonCreator + public MetastoreStmt( + @JsonProperty("sql") String sql + ) + { + this.sql = sql; + } + + @JsonProperty("sql") + public String sql() + { + return sql; + } + + @Override + public String toString() + { + return TestConfigs.toYaml(this); + } + + /** + * Convert the human-readable form of the statement in YAML + * into the compact JSON form preferred in the DB. Also + * compacts the SQL, but that's OK. + */ + public String toSQL() + { + String stmt = RegExUtils.replaceAll(sql, "\n", " "); + stmt = RegExUtils.replaceAll(stmt, " +", " "); + stmt = RegExUtils.replaceAll(stmt, ": ", ":"); + stmt = RegExUtils.replaceAll(stmt, ", ", ","); + stmt = RegExUtils.replaceAll(stmt, " }", "}"); + stmt = RegExUtils.replaceAll(stmt, "\\{ ", "{"); + return stmt; + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ResolvedConfig.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ResolvedConfig.java new file mode 100644 index 00000000000..6bdfe96b2f9 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ResolvedConfig.java @@ -0,0 +1,406 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.config; + +import com.google.common.base.Strings; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.curator.CuratorConfig; +import org.apache.druid.curator.ExhibitorConfig; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.testing.IntegrationTestingConfigProvider; +import org.apache.druid.testsEx.config.ClusterConfig.ClusterType; +import org.apache.druid.testsEx.config.ResolvedService.ResolvedKafka; +import org.apache.druid.testsEx.config.ResolvedService.ResolvedZk; +import org.apache.druid.testsEx.config.ServiceConfig.DruidConfig; + +import java.io.BufferedReader; +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStreamReader; +import java.nio.charset.StandardCharsets; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Map.Entry; + +public class ResolvedConfig +{ + public static final String COORDINATOR = "coordinator"; + public static final String HISTORICAL = "historical"; + public static final String OVERLORD = "overlord"; + public static final String BROKER = "broker"; + public static final String ROUTER = "router"; + public static final String MIDDLEMANAGER = "middlemanager"; + public static final String INDEXER = "indexer"; + + public static final int DEFAULT_READY_TIMEOUT_SEC = 120; + public static final int DEFAULT_READY_POLL_MS = 2000; + + private final String category; + private final ClusterType type; + private final String proxyHost; + private final int readyTimeoutSec; + private final int readyPollMs; + final String datasourceNameSuffix; + private Map properties; + private Map settings; + + private final ResolvedZk zk; + private final ResolvedKafka kafka; + private final ResolvedMetastore metastore; + private final Map druidServices = new HashMap<>(); + + public ResolvedConfig(String category, ClusterConfig config) + { + this.category = category; + type = config.type() == null ? ClusterType.docker : config.type(); + if (!hasProxy()) { + proxyHost = null; + } else if (Strings.isNullOrEmpty(config.proxyHost())) { + proxyHost = "localhost"; + } else { + proxyHost = config.proxyHost(); + } + readyTimeoutSec = config.readyTimeoutSec() > 0 ? + config.readyTimeoutSec() : DEFAULT_READY_TIMEOUT_SEC; + readyPollMs = config.readyPollMs() > 0 ? config.readyPollMs() : DEFAULT_READY_POLL_MS; + if (config.properties() == null) { + this.properties = ImmutableMap.of(); + } else { + this.properties = config.properties(); + } + if (config.settings() == null) { + this.settings = ImmutableMap.of(); + } else { + this.settings = config.settings(); + } + if (config.datasourceSuffix() == null) { + this.datasourceNameSuffix = ""; + } else { + this.datasourceNameSuffix = config.datasourceSuffix(); + } + + if (config.zk() == null) { + this.zk = null; + } else { + this.zk = new ResolvedZk(this, config.zk()); + } + if (config.kafka() == null) { + this.kafka = null; + } else { + this.kafka = new ResolvedKafka(this, config.kafka()); + } + if (config.metastore() == null) { + this.metastore = null; + } else { + this.metastore = new ResolvedMetastore(this, config.metastore(), config); + } + + if (config.druid() != null) { + for (Entry entry : config.druid().entrySet()) { + druidServices.put(entry.getKey(), + new ResolvedDruidService(this, entry.getValue(), entry.getKey())); + } + } + } + + public ClusterType type() + { + return type; + } + + public String proxyHost() + { + return proxyHost; + } + + public int readyTimeoutSec() + { + return readyTimeoutSec; + } + + public int readyPollMs() + { + return readyPollMs; + } + + public boolean isDocker() + { + return type == ClusterType.docker; + } + + public boolean hasProxy() + { + switch (type) { + case docker: + case k8s: + return true; + default: + return false; + } + } + + public ResolvedZk zk() + { + return zk; + } + + public ResolvedMetastore metastore() + { + return metastore; + } + + public ResolvedKafka kafka() + { + return kafka; + } + + public Map settings() + { + return settings; + } + + public Map properties() + { + return properties; + } + + public Map requireDruid() + { + if (druidServices == null) { + throw new ISE("Please configure Druid services"); + } + return druidServices; + } + + public ResolvedZk requireZk() + { + if (zk == null) { + throw new ISE("Please specify the ZooKeeper configuration"); + } + return zk; + } + + public ResolvedMetastore requireMetastore() + { + if (metastore == null) { + throw new ISE("Please specify the Metastore configuration"); + } + return metastore; + } + + public ResolvedKafka requireKafka() + { + if (kafka == null) { + throw new ISE("Please specify the Kafka configuration"); + } + return kafka; + } + + public ResolvedDruidService druidService(String serviceKey) + { + return requireDruid().get(serviceKey); + } + + public ResolvedDruidService requireService(String serviceKey) + { + ResolvedDruidService service = druidService(serviceKey); + if (service == null) { + throw new ISE("Please configure Druid service " + serviceKey); + } + return service; + } + + public ResolvedDruidService requireCoordinator() + { + return requireService(COORDINATOR); + } + + public ResolvedDruidService requireOverlord() + { + return requireService(OVERLORD); + } + + public ResolvedDruidService requireBroker() + { + return requireService(BROKER); + } + + public ResolvedDruidService requireRouter() + { + return requireService(ROUTER); + } + + public ResolvedDruidService requireMiddleManager() + { + return requireService(MIDDLEMANAGER); + } + + public ResolvedDruidService requireHistorical() + { + return requireService(HISTORICAL); + } + + public String routerUrl() + { + return requireRouter().clientUrl(); + } + + public CuratorConfig toCuratorConfig() + { + if (zk == null) { + throw new ISE("ZooKeeper not configured"); + } + // TODO: Add a builder for other properties + return CuratorConfig.create(zk.clientHosts()); + } + + public ExhibitorConfig toExhibitorConfig() + { + // Does not yet support exhibitors + return ExhibitorConfig.create(Collections.emptyList()); + } + + /** + * Map from old-style config file (and settings) name to the + * corresponding property. + */ + private static final Map SETTINGS_MAP = + ImmutableMap.builder() + .put("cloud_bucket", "cloudBucket") + .put("cloud_path", "cloudPath") + .put("cloud_region", "cloudRegion") + .put("s3_assume_role_with_external_id", "s3AssumeRoleWithExternalId") + .put("s3_assume_role_external_id", "s3AssumeRoleExternalId") + .put("s3_assume_role_without_external_id", "s3AssumeRoleWithoutExternalId") + .put("stream_endpoint", "streamEndpoint") + .put("s3_accessKey", "s3AccessKey") + .put("s3_secretKey", "s3SecretKey") + .put("azure_account", "azureAccount") + .put("azure_key", "azureKey") + .put("azure_container", "azureContainer") + .put("google_bucket", "googleBucket") + .put("google_prefix", "googlePrefix") + .build(); + + private static void setDruidProperyVar(Map properties, String key, Object value) + { + if (value == null) { + return; + } + if (key.startsWith("druid_")) { + key = key.substring("druid_".length()); + } + String mapped = SETTINGS_MAP.get(key); + key = mapped == null ? key : mapped; + TestConfigs.putProperty(properties, IntegrationTestingConfigProvider.PROPERTY_BASE, key, value.toString()); + } + + private void loadPropertyFile(Map properties, File file) + { + try (BufferedReader in = new BufferedReader( + new InputStreamReader(new FileInputStream(file), StandardCharsets.UTF_8))) { + String line; + while ((line = in.readLine()) != null) { + if (Strings.isNullOrEmpty(line) || line.startsWith("#")) { + continue; + } + String[] parts = line.split("="); + if (parts.length != 2) { + continue; + } + setDruidProperyVar(properties, parts[0], parts[1]); + } + } + catch (IOException e) { + throw new IAE(e, "Cannot read file %s", file.getAbsolutePath()); + } + } + + /** + * Convert the config in this structure the the properties + * used to configure Guice. + */ + public Map toProperties() + { + Map properties = new HashMap<>(); + // druid.test.config.dockerIp is used by some older test code. Remove + // it when that code is updated. + TestConfigs.putProperty(properties, "druid.test.config.dockerIp", proxyHost); + + // Start with implicit properties from various sections. + if (zk != null) { + properties.putAll(zk.toProperties()); + } + if (metastore != null) { + properties.putAll(metastore.toProperties()); + } + + // Add settings, converted to properties. Map both old and + // "property-style" settings to the full property path. + // Settings are converted to properties so they can be overridden + // by environment variables and -D command-line settings. + for (Map.Entry entry : settings.entrySet()) { + setDruidProperyVar(properties, entry.getKey(), entry.getValue()); + } + + // Add explicit properties + if (this.properties != null) { + properties.putAll(this.properties); + } + + // Override with a user-specific config file. + File userEnv = new File( + new File( + System.getProperty("user.home"), + "druid-it"), + category + ".env"); + if (userEnv.exists()) { + loadPropertyFile(properties, userEnv); + } + + // Override with a user-specific config file. + String overrideEnv = System.getenv("OVERRIDE_ENV"); + if (overrideEnv != null) { + loadPropertyFile(properties, new File(overrideEnv)); + } + + // Override with any environment variables of the form "druid_" + for (Map.Entry entry : System.getenv().entrySet()) { + String key = entry.getKey(); + if (!key.startsWith("druid_")) { + continue; + } + setDruidProperyVar(properties, key, entry.getValue()); + } + + // Override with any system properties of the form "druid_" + for (Map.Entry entry : System.getProperties().entrySet()) { + String key = (String) entry.getKey(); + if (!key.startsWith("druid_")) { + continue; + } + setDruidProperyVar(properties, key, entry.getValue()); + } + return properties; + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ResolvedDruidService.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ResolvedDruidService.java new file mode 100644 index 00000000000..58cbdcdc33c --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ResolvedDruidService.java @@ -0,0 +1,145 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.config; + +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.testsEx.config.ServiceConfig.DruidConfig; + +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +public class ResolvedDruidService extends ResolvedService +{ + public ResolvedDruidService(ResolvedConfig root, DruidConfig config, String serviceKey) + { + super(root, config, serviceKey); + } + + /** + * Get the URL (visible to the test) of the service. + */ + public String clientUrl() + { + return resolveUrl(instance()); + } + + /** + * Find an instance given the instance name (tag). + */ + public ResolvedInstance findInstance(String instanceName) + { + for (ResolvedInstance instance : requireInstances()) { + if (instance.tag() != null && instance.tag().equals(instanceName)) { + return instance; + } + } + return null; + } + + /** + * Find an instance given the instance name (tag). Raises + * an error (which fails the test) if the tag is not defined. + */ + public ResolvedInstance requireInstance(String instanceName) + { + ResolvedInstance instance = findInstance(instanceName); + if (instance != null) { + return instance; + } + throw new ISE( + StringUtils.format( + "No Druid instance of service %s with name %s is defined", + service, + instanceName)); + } + + public String resolveUrl() + { + return resolveUrl(instance()); + } + + /** + * Return the URL for the given instance name (tag) of this service + * as visible to the test. + */ + public String resolveUrl(String instanceName) + { + return resolveUrl(requireInstance(instanceName)); + } + + /** + * Return the URL, known to the test, of the given service instance. + */ + public String resolveUrl(ResolvedInstance instance) + { + return StringUtils.format( + "http://%s:%d", + instance.clientHost(), + instance.clientPort()); + } + + /** + * Return the named service instance. If not found, return the + * "default" instance. This is used by the somewhat awkward test + * config object so that if a test asks for "Coordinator one" in + * a cluster with a single Coordinator, it will get that Coordinator. + * Same for Overlord. + */ + public ResolvedInstance tagOrDefault(String tag) + { + ResolvedInstance taggedInstance = findInstance(tag); + return taggedInstance == null ? instance() : taggedInstance; + } + + /** + * Returns the "default" host for this service as known to the + * cluster. The host is that of the only instance and is undefined + * if there are multiple instances. + */ + public String resolveHost() + { + ResolvedInstance instance = instance(); + if (instances.size() > 1) { + throw new ISE( + StringUtils.format("Service %s has %d hosts, default is ambiguous", + service, + instances.size())); + } + return instance.host(); + } + + public ResolvedInstance findHost(String host) + { + Pattern p = Pattern.compile("https?://(.*):(\\d+)"); + Matcher m = p.matcher(host); + if (!m.matches()) { + return null; + } + String hostName = m.group(1); + int port = Integer.parseInt(m.group(2)); + for (ResolvedInstance instance : instances) { + if (instance.host().equals(hostName) && instance.port() == port) { + return instance; + } + } + return null; + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ResolvedMetastore.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ResolvedMetastore.java new file mode 100644 index 00000000000..f65790eb1c9 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ResolvedMetastore.java @@ -0,0 +1,128 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.config; + +import com.google.common.collect.ImmutableMap; +import org.apache.commons.lang3.RegExUtils; +import org.apache.druid.metadata.MetadataStorageConnectorConfig; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class ResolvedMetastore extends ResolvedService +{ + // Set to be 1 sec. longer than the setting in the + // docker-compose.yaml file: + // druid_manager_segments_pollDuration=PT5S + public static final int DEFAULT_METASTORE_INIT_DELAY_SEC = 6; + + private final String driver; + private final String connectURI; + private final String user; + private final String password; + private final Map properties; + private final int initDelaySec; + private List initStmts; + + public ResolvedMetastore(ResolvedConfig root, MetastoreConfig config, ClusterConfig clusterConfig) + { + super(root, config, "metastore"); + this.driver = config.driver(); + if (config.connectURI() != null) { + ResolvedInstance instance = instance(); + this.connectURI = RegExUtils.replaceAll( + RegExUtils.replaceAll( + config.connectURI(), + "", + Integer.toString(instance.clientPort())), + "", + instance.clientHost()); + } else { + this.connectURI = null; + } + this.user = config.user(); + this.password = config.password(); + if (config.properties() == null) { + this.properties = ImmutableMap.of(); + } else { + this.properties = config.properties(); + } + + this.initDelaySec = clusterConfig.metastoreInitDelaySec() > 0 + ? clusterConfig.metastoreInitDelaySec() + : DEFAULT_METASTORE_INIT_DELAY_SEC; + this.initStmts = clusterConfig.metastoreInit(); + } + + public String driver() + { + return driver; + } + + public String connectURI() + { + return connectURI; + } + + public String user() + { + return user; + } + + public String password() + { + return password; + } + + public Map properties() + { + return properties; + } + + /** + * Create the properties Guice needs to create the connector config. + * + * @see + * Setting up MySQL + */ + public Map toProperties() + { + final String base = MetadataStorageConnectorConfig.PROPERTY_BASE; + Map properties = new HashMap<>(); + TestConfigs.putProperty(properties, "druid.metadata.mysql.driver.driverClassName", driver); + TestConfigs.putProperty(properties, "druid.metadata.storage.type", "mysql"); + TestConfigs.putProperty(properties, base, "connectURI", connectURI); + TestConfigs.putProperty(properties, base, "user", user); + TestConfigs.putProperty(properties, base, "password", password); + TestConfigs.putProperty(properties, base, "dbcp", this.properties); + return properties; + } + + public List initStmts() + { + return initStmts; + } + + public int initDelaySec() + { + return initDelaySec; + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ResolvedService.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ResolvedService.java new file mode 100644 index 00000000000..5bd377f7079 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ResolvedService.java @@ -0,0 +1,219 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.config; + +import com.google.common.collect.ImmutableMap; +import org.apache.druid.curator.CuratorConfig; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.testsEx.config.ServiceConfig.ZKConfig; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +public class ResolvedService +{ + protected final String service; + protected final List instances = new ArrayList<>(); + + public ResolvedService(ResolvedConfig root, ServiceConfig config, String name) + { + this.service = config.service() == null ? name : config.service(); + for (ServiceInstance instanceConfig : config.instances()) { + this.instances.add(new ResolvedInstance(root, instanceConfig, this)); + } + } + + public String service() + { + return service; + } + + public List requireInstances() + { + if (instances.isEmpty()) { + throw new ISE("Please specify a " + service + " instance"); + } + return instances; + } + + public ResolvedInstance instance() + { + return requireInstances().get(0); + } + + public class ResolvedInstance + { + private final String container; + private final String host; + private final String clientHost; + private final String tag; + private final int port; + private final int clientPort; + + public ResolvedInstance(ResolvedConfig root, ServiceInstance config, ResolvedService service) + { + this.tag = config.tag(); + + // The actual (cluster) host is... + if (config.host() != null) { + // The specified host, if provided + this.host = config.host(); + } else { + String baseHost; + if (root.hasProxy()) { + // The same as the service, if there is a proxy + baseHost = service.service; + // with the tag appended + if (tag != null) { + baseHost += "-" + config.tag(); + } + this.host = baseHost; + } else { + // The local host otherwise + this.host = "localhost"; + } + } + + if (root.hasProxy()) { + this.clientHost = root.proxyHost(); + } else { + this.clientHost = this.host; + } + + this.container = config.container() != null ? config.container() : service.service; + if (config.port() == 0) { + throw new ISE("Must provide port"); + } + this.port = config.port(); + if (config.proxyPort() != 0) { + this.clientPort = config.proxyPort(); + } else { + this.clientPort = this.port; + } + } + + public ResolvedService service() + { + return ResolvedService.this; + } + + public String container() + { + return container; + } + + public String host() + { + return host; + } + + public String clientHost() + { + return clientHost; + } + + public String tag() + { + return tag; + } + + public int port() + { + return port; + } + + public int clientPort() + { + return clientPort; + } + } + + public static class ResolvedZk extends ResolvedService + { + private final int startTimeoutSecs; + + public ResolvedZk(ResolvedConfig root, ZKConfig config) + { + super(root, config, "zookeeper"); + this.startTimeoutSecs = config.startTimeoutSecs(); + } + + public int startTimeoutSecs() + { + return startTimeoutSecs; + } + + public String clientHosts() + { + List hosts = new ArrayList<>(); + for (ResolvedInstance instance : instances) { + hosts.add(formatHost(instance.clientHost(), instance.clientPort())); + } + return String.join(",", hosts); + } + + public String clusterHosts() + { + List hosts = new ArrayList<>(); + for (ResolvedInstance instance : instances) { + hosts.add(formatHost(instance.host(), instance.port())); + } + return String.join(",", hosts); + } + + private String formatHost(String host, int port) + { + return StringUtils.format("%s:%d", host, port); + } + + public Map toProperties() + { + /* + * We will use this instead of druid server's CuratorConfig, because CuratorConfig in + * a test cluster environment sees Zookeeper at localhost even if Zookeeper is elsewhere. + * We'll take the Zookeeper host from the configuration file instead. + */ + return ImmutableMap.of( + CuratorConfig.CONFIG_PREFIX + ".zkHosts", + clientHosts()); + } + } + + public static class ResolvedKafka extends ResolvedService + { + public ResolvedKafka(ResolvedConfig root, KafkaConfig config) + { + super(root, config, "kafka"); + } + + public String clientHost() + { + return instance().clientHost(); + } + + public String bootstrap() + { + ResolvedInstance instance = instance(); + return StringUtils.format("%s:%d", instance.clientHost(), instance.clientPort()); + } + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ServiceConfig.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ServiceConfig.java new file mode 100644 index 00000000000..08b07bc4ed8 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ServiceConfig.java @@ -0,0 +1,116 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.config; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonInclude.Include; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.List; + +public class ServiceConfig +{ + protected final String service; + protected List instances; + + public ServiceConfig( + String service, + List instances + ) + { + this.service = service; + this.instances = instances; + } + + @JsonProperty("service") + @JsonInclude(Include.NON_NULL) + public String service() + { + return service; + } + + @JsonProperty("instances") + @JsonInclude(Include.NON_NULL) + public List instances() + { + return instances; + } + + @Override + public String toString() + { + return TestConfigs.toYaml(this); + } + + /** + * YAML description of a ZK cluster. Converted to + * {@link org.apache.druid.curator.CuratorConfig} + */ + public static class ZKConfig extends ServiceConfig + { + /** + * Amount of time to wait for ZK to become ready. + * Defaults to 5 seconds. + */ + private final int startTimeoutSecs; + + @JsonCreator + public ZKConfig( + @JsonProperty("service") String service, + @JsonProperty("startTimeoutSecs") int startTimeoutSecs, + @JsonProperty("instances") List instances + ) + { + super(service, instances); + this.startTimeoutSecs = startTimeoutSecs; + } + + @JsonProperty("startTimeoutSecs") + public int startTimeoutSecs() + { + return startTimeoutSecs; + } + } + + /** + * Represents a Druid service (of one or more instances) running + * in the test cluster. The service name comes from the key used + * in the {@code druid} map:

+   * druid:
+   *   broker:  # <-- key (service name)
+   *     instances:
+   *       ...
+   * 
+ */ + public static class DruidConfig extends ServiceConfig + { + @JsonCreator + public DruidConfig( + // Note: service is not actually used. + @JsonProperty("service") String service, + @JsonProperty("instances") List instances + ) + { + super(service, instances); + } + } + +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ServiceInstance.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ServiceInstance.java new file mode 100644 index 00000000000..c9a7e0e3a87 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/ServiceInstance.java @@ -0,0 +1,124 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.config; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonInclude.Include; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * One instance of a Druid or third-party service running on + * a host or in a container. + */ +public class ServiceInstance +{ + /** + * Name of the Docker container. Used in Docker commands against + * the container, such as starting and stopping. + */ + private final String container; + + /** + * Name of the host running the service as known to the cluster + * (which many not be visible to the host running the test.) + * Assumed to be {@code } or @{code -} + * if not explicitly set. + */ + private final String host; + + /** + * Tag used to identify a service when there are multiple + * instances. The host is assumed to be @{code-} if + * not explicitly set. + */ + private final String tag; + + /** + * The port exposed by the service on its host. May not be + * visible to the test. Required. + */ + + private final int port; + + /** + * The proxy port visible for the test for this service. Defaults + * to the same as the @{code port}. Define only if Docker is configured + * for port mapping other than identity. + */ + private final int proxyPort; + + @JsonCreator + public ServiceInstance( + @JsonProperty("container") String container, + @JsonProperty("host") String host, + @JsonProperty("tag") String tag, + @JsonProperty("port") int port, + @JsonProperty("proxyPort") int proxyPort + ) + { + this.container = container; + this.host = host; + this.tag = tag; + this.port = port; + this.proxyPort = proxyPort; + } + + @JsonProperty("container") + @JsonInclude(Include.NON_NULL) + public String container() + { + return container; + } + + @JsonProperty("host") + @JsonInclude(Include.NON_NULL) + public String host() + { + return host; + } + + @JsonProperty("tag") + @JsonInclude(Include.NON_NULL) + public String tag() + { + return tag; + } + + @JsonProperty("port") + @JsonInclude(Include.NON_DEFAULT) + public int port() + { + return port; + } + + @JsonProperty("proxyPort") + @JsonInclude(Include.NON_DEFAULT) + public int proxyPort() + { + return proxyPort; + } + + @Override + public String toString() + { + return TestConfigs.toYaml(this); + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/TestConfigs.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/TestConfigs.java new file mode 100644 index 00000000000..8b909487c23 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/TestConfigs.java @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.config; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.dataformat.yaml.YAMLFactory; +import com.fasterxml.jackson.dataformat.yaml.YAMLGenerator; + +import java.util.Map; + +/** + * Utility functions related to test configuration. + */ +public class TestConfigs +{ + /** + * Converts a YAML-aware object to a YAML string, primarily + * for use in @{code toString()} methods. + */ + public static String toYaml(Object obj) + { + ObjectMapper mapper = new ObjectMapper( + new YAMLFactory() + .enable(YAMLGenerator.Feature.MINIMIZE_QUOTES)); + try { + return mapper.writeValueAsString(obj); + } + catch (JsonProcessingException e) { + return ""; + } + } + + public static void putProperty(Map properties, String key, Object value) + { + if (value == null) { + return; + } + properties.put(key, value); + } + + public static void putProperty(Map properties, String base, String key, Object value) + { + if (value == null) { + return; + } + properties.put(base + "." + key, value); + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/AbstractITBatchIndexTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/AbstractITBatchIndexTest.java new file mode 100644 index 00000000000..5b080cd7448 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/AbstractITBatchIndexTest.java @@ -0,0 +1,502 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.indexer; + +import com.google.common.collect.FluentIterable; +import com.google.inject.Inject; +import org.apache.commons.io.IOUtils; +import org.apache.druid.indexer.partitions.SecondaryPartitionType; +import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReport; +import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData; +import org.apache.druid.indexing.common.TaskReport; +import org.apache.druid.indexing.common.task.batch.parallel.PartialDimensionCardinalityTask; +import org.apache.druid.indexing.common.task.batch.parallel.PartialDimensionDistributionTask; +import org.apache.druid.indexing.common.task.batch.parallel.PartialGenericSegmentMergeTask; +import org.apache.druid.indexing.common.task.batch.parallel.PartialHashSegmentGenerateTask; +import org.apache.druid.indexing.common.task.batch.parallel.PartialRangeSegmentGenerateTask; +import org.apache.druid.indexing.common.task.batch.parallel.SinglePhaseSubTask; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.testing.IntegrationTestingConfig; +import org.apache.druid.testing.clients.ClientInfoResourceTestClient; +import org.apache.druid.testing.utils.ITRetryUtil; +import org.apache.druid.testing.utils.SqlTestQueryHelper; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.TimelineObjectHolder; +import org.apache.druid.timeline.VersionedIntervalTimeline; +import org.junit.Assert; + +import java.io.IOException; +import java.io.InputStream; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.List; +import java.util.function.Function; + +public abstract class AbstractITBatchIndexTest extends AbstractIndexerTest +{ + public enum InputFormatDetails + { + AVRO("avro_ocf", ".avro", "/avro"), + CSV("csv", ".csv", "/csv"), + TSV("tsv", ".tsv", "/tsv"), + ORC("orc", ".orc", "/orc"), + JSON("json", ".json", "/json"), + PARQUET("parquet", ".parquet", "/parquet"); + + private final String inputFormatType; + private final String fileExtension; + private final String folderSuffix; + + InputFormatDetails(String inputFormatType, String fileExtension, String folderSuffix) + { + this.inputFormatType = inputFormatType; + this.fileExtension = fileExtension; + this.folderSuffix = folderSuffix; + } + + public String getInputFormatType() + { + return inputFormatType; + } + + public String getFileExtension() + { + return fileExtension; + } + + public String getFolderSuffix() + { + return folderSuffix; + } + } + + private static final Logger LOG = new Logger(AbstractITBatchIndexTest.class); + + @Inject + protected IntegrationTestingConfig config; + @Inject + protected SqlTestQueryHelper sqlQueryHelper; + + @Inject + ClientInfoResourceTestClient clientInfoResourceTestClient; + + protected void doIndexTest( + String dataSource, + String indexTaskFilePath, + String queryFilePath, + boolean waitForNewVersion, + boolean runTestQueries, + boolean waitForSegmentsToLoad, + Pair segmentAvailabilityConfirmationPair + ) throws IOException + { + doIndexTest( + dataSource, + indexTaskFilePath, + Function.identity(), + queryFilePath, + waitForNewVersion, + runTestQueries, + waitForSegmentsToLoad, + segmentAvailabilityConfirmationPair + ); + } + + protected void doIndexTest( + String dataSource, + String indexTaskFilePath, + Function taskSpecTransform, + String queryFilePath, + boolean waitForNewVersion, + boolean runTestQueries, + boolean waitForSegmentsToLoad, + Pair segmentAvailabilityConfirmationPair + ) throws IOException + { + final String fullDatasourceName = dataSource + config.getExtraDatasourceNameSuffix(); + final String taskSpec = taskSpecTransform.apply( + StringUtils.replace( + getResourceAsString(indexTaskFilePath), + "%%DATASOURCE%%", + fullDatasourceName + ) + ); + + submitTaskAndWait( + taskSpec, + fullDatasourceName, + waitForNewVersion, + waitForSegmentsToLoad, + segmentAvailabilityConfirmationPair + ); + if (runTestQueries) { + doTestQuery(dataSource, queryFilePath); + } + } + + protected void doTestQuery(String dataSource, String queryFilePath) + { + try { + String queryResponseTemplate; + try { + InputStream is = AbstractITBatchIndexTest.class.getResourceAsStream(queryFilePath); + queryResponseTemplate = IOUtils.toString(is, StandardCharsets.UTF_8); + } + catch (IOException e) { + throw new ISE(e, "could not read query file: %s", queryFilePath); + } + + queryResponseTemplate = StringUtils.replace( + queryResponseTemplate, + "%%DATASOURCE%%", + dataSource + config.getExtraDatasourceNameSuffix() + ); + queryHelper.testQueriesFromString(queryResponseTemplate); + + } + catch (Exception e) { + LOG.error(e, "Error while testing"); + throw new RuntimeException(e); + } + } + + protected void doReindexTest( + String baseDataSource, + String reindexDataSource, + String reindexTaskFilePath, + String queryFilePath, + Pair segmentAvailabilityConfirmationPair + ) throws IOException + { + doReindexTest( + baseDataSource, + reindexDataSource, + Function.identity(), + reindexTaskFilePath, + queryFilePath, + segmentAvailabilityConfirmationPair + ); + } + + void doReindexTest( + String baseDataSource, + String reindexDataSource, + Function taskSpecTransform, + String reindexTaskFilePath, + String queryFilePath, + Pair segmentAvailabilityConfirmationPair + ) throws IOException + { + final String fullBaseDatasourceName = baseDataSource + config.getExtraDatasourceNameSuffix(); + final String fullReindexDatasourceName = reindexDataSource + config.getExtraDatasourceNameSuffix(); + + String taskSpec = StringUtils.replace( + getResourceAsString(reindexTaskFilePath), + "%%DATASOURCE%%", + fullBaseDatasourceName + ); + + taskSpec = StringUtils.replace( + taskSpec, + "%%REINDEX_DATASOURCE%%", + fullReindexDatasourceName + ); + + taskSpec = taskSpecTransform.apply(taskSpec); + + submitTaskAndWait( + taskSpec, + fullReindexDatasourceName, + false, + true, + segmentAvailabilityConfirmationPair + ); + try { + String queryResponseTemplate; + try { + InputStream is = AbstractITBatchIndexTest.class.getResourceAsStream(queryFilePath); + queryResponseTemplate = IOUtils.toString(is, StandardCharsets.UTF_8); + } + catch (IOException e) { + throw new ISE(e, "could not read query file: %s", queryFilePath); + } + + queryResponseTemplate = StringUtils.replace( + queryResponseTemplate, + "%%DATASOURCE%%", + fullReindexDatasourceName + ); + + queryHelper.testQueriesFromString(queryResponseTemplate); + // verify excluded dimension is not reIndexed + final List dimensions = clientInfoResourceTestClient.getDimensions( + fullReindexDatasourceName, + "2013-08-31T00:00:00.000Z/2013-09-10T00:00:00.000Z" + ); + Assert.assertFalse("dimensions : " + dimensions, dimensions.contains("robot")); + } + catch (Exception e) { + LOG.error(e, "Error while testing"); + throw new RuntimeException(e); + } + } + + void doIndexTestSqlTest( + String dataSource, + String indexTaskFilePath, + String queryFilePath + ) throws IOException + { + doIndexTestSqlTest( + dataSource, + indexTaskFilePath, + queryFilePath, + Function.identity() + ); + } + void doIndexTestSqlTest( + String dataSource, + String indexTaskFilePath, + String queryFilePath, + Function taskSpecTransform + ) throws IOException + { + final String fullDatasourceName = dataSource + config.getExtraDatasourceNameSuffix(); + final String taskSpec = taskSpecTransform.apply( + StringUtils.replace( + getResourceAsString(indexTaskFilePath), + "%%DATASOURCE%%", + fullDatasourceName + ) + ); + + Pair dummyPair = new Pair<>(false, false); + submitTaskAndWait(taskSpec, fullDatasourceName, false, true, dummyPair); + try { + sqlQueryHelper.testQueriesFromFile(queryFilePath); + } + catch (Exception e) { + LOG.error(e, "Error while testing"); + throw new RuntimeException(e); + } + } + + protected void submitTaskAndWait( + String taskSpec, + String dataSourceName, + boolean waitForNewVersion, + boolean waitForSegmentsToLoad, + Pair segmentAvailabilityConfirmationPair + ) + { + final List oldVersions = waitForNewVersion ? coordinator.getAvailableSegments(dataSourceName) : null; + + long startSubTaskCount = -1; + final boolean assertRunsSubTasks = taskSpec.contains("index_parallel"); + if (assertRunsSubTasks) { + startSubTaskCount = countCompleteSubTasks(dataSourceName, !taskSpec.contains("dynamic")); + } + + final String taskID = indexer.submitTask(taskSpec); + LOG.info("TaskID for loading index task %s", taskID); + indexer.waitUntilTaskCompletes(taskID); + + if (assertRunsSubTasks) { + final boolean perfectRollup = !taskSpec.contains("dynamic"); + final long newSubTasks = countCompleteSubTasks(dataSourceName, perfectRollup) - startSubTaskCount; + Assert.assertTrue( + StringUtils.format( + "The supervisor task [%s] didn't create any sub tasks. Was it executed in the parallel mode?", + taskID + ), + newSubTasks > 0 + ); + } + + if (segmentAvailabilityConfirmationPair.lhs != null && segmentAvailabilityConfirmationPair.lhs) { + TaskReport reportRaw = indexer.getTaskReport(taskID).get("ingestionStatsAndErrors"); + IngestionStatsAndErrorsTaskReport report = (IngestionStatsAndErrorsTaskReport) reportRaw; + IngestionStatsAndErrorsTaskReportData reportData = (IngestionStatsAndErrorsTaskReportData) report.getPayload(); + + // Confirm that the task waited longer than 0ms for the task to complete. + Assert.assertTrue(reportData.getSegmentAvailabilityWaitTimeMs() > 0); + + // Make sure that the result of waiting for segments to load matches the expected result + if (segmentAvailabilityConfirmationPair.rhs != null) { + Assert.assertEquals( + Boolean.valueOf(reportData.isSegmentAvailabilityConfirmed()), + segmentAvailabilityConfirmationPair.rhs + ); + } + } + + // IT*ParallelIndexTest do a second round of ingestion to replace segments in an existing + // data source. For that second round we need to make sure the coordinator actually learned + // about the new segments before waiting for it to report that all segments are loaded; otherwise + // this method could return too early because the coordinator is merely reporting that all the + // original segments have loaded. + if (waitForNewVersion) { + ITRetryUtil.retryUntilTrue( + () -> { + final VersionedIntervalTimeline timeline = VersionedIntervalTimeline.forSegments( + coordinator.getAvailableSegments(dataSourceName) + ); + + final List> holders = timeline.lookup(Intervals.ETERNITY); + return FluentIterable + .from(holders) + .transformAndConcat(TimelineObjectHolder::getObject) + .anyMatch( + chunk -> FluentIterable.from(oldVersions) + .anyMatch(oldSegment -> chunk.getObject().overshadows(oldSegment)) + ); + }, + "See a new version" + ); + } + + if (waitForSegmentsToLoad) { + ITRetryUtil.retryUntilTrue( + () -> coordinator.areSegmentsLoaded(dataSourceName), "Segment Load" + ); + } + } + + private long countCompleteSubTasks(final String dataSource, final boolean perfectRollup) + { + return indexer.getCompleteTasksForDataSource(dataSource) + .stream() + .filter(t -> { + if (!perfectRollup) { + return t.getType().equals(SinglePhaseSubTask.TYPE); + } else { + return t.getType().equalsIgnoreCase(PartialHashSegmentGenerateTask.TYPE) + || t.getType().equalsIgnoreCase(PartialDimensionDistributionTask.TYPE) + || t.getType().equalsIgnoreCase(PartialDimensionCardinalityTask.TYPE) + || t.getType().equalsIgnoreCase(PartialRangeSegmentGenerateTask.TYPE) + || t.getType().equalsIgnoreCase(PartialGenericSegmentMergeTask.TYPE); + } + }) + .count(); + } + + void verifySegmentsCountAndLoaded(String dataSource, int numExpectedSegments) + { + ITRetryUtil.retryUntilTrue( + () -> coordinator.areSegmentsLoaded(dataSource + config.getExtraDatasourceNameSuffix()), + "Segment load check" + ); + ITRetryUtil.retryUntilTrue( + () -> { + List segments = coordinator.getAvailableSegments( + dataSource + config.getExtraDatasourceNameSuffix() + ); + int segmentCount = segments.size(); + LOG.info("Current segment count: %d, expected: %d", segmentCount, numExpectedSegments); + + return segmentCount == numExpectedSegments; + }, + "Segment count check" + ); + } + + void verifySegmentsCountAndLoaded(String dataSource, int numExpectedSegments, int numExpectedTombstones) + { + ITRetryUtil.retryUntilTrue( + () -> coordinator.areSegmentsLoaded(dataSource + config.getExtraDatasourceNameSuffix()), + "Segment load check" + ); + ITRetryUtil.retryUntilTrue( + () -> { + List segments = coordinator.getAvailableSegments( + dataSource + config.getExtraDatasourceNameSuffix() + ); + int segmentCount = segments.size(); + LOG.info("Current segment count: %d, expected: %d", segmentCount, numExpectedSegments); + + int tombstoneCount = 0; + for (DataSegment segment : segments) { + if (segment.isTombstone()) { + tombstoneCount++; + } + } + + LOG.info("Current tombstone count: %d, expected: %d", tombstoneCount, numExpectedTombstones); + + return segmentCount == numExpectedSegments && tombstoneCount == numExpectedTombstones; + }, + "Segment count check" + ); + } + + void compactData(String dataSource, String compactionTask) throws Exception + { + final String fullDatasourceName = dataSource + config.getExtraDatasourceNameSuffix(); + final List intervalsBeforeCompaction = coordinator.getSegmentIntervals(fullDatasourceName); + intervalsBeforeCompaction.sort(null); + final String template = getResourceAsString(compactionTask); + String taskSpec = StringUtils.replace(template, "%%DATASOURCE%%", fullDatasourceName); + + final String taskID = indexer.submitTask(taskSpec); + LOG.info("TaskID for compaction task %s", taskID); + indexer.waitUntilTaskCompletes(taskID); + + ITRetryUtil.retryUntilTrue( + () -> coordinator.areSegmentsLoaded(fullDatasourceName), + "Segment Compaction" + ); + ITRetryUtil.retryUntilTrue( + () -> { + final List actualIntervals = coordinator.getSegmentIntervals( + dataSource + config.getExtraDatasourceNameSuffix() + ); + actualIntervals.sort(null); + return actualIntervals.equals(intervalsBeforeCompaction); + }, + "Compaction interval check" + ); + } + + void verifySegmentsCompacted(String dataSource, int expectedCompactedSegmentCount) + { + List segments = coordinator.getFullSegmentsMetadata( + dataSource + config.getExtraDatasourceNameSuffix() + ); + List foundCompactedSegments = new ArrayList<>(); + for (DataSegment segment : segments) { + if (segment.getLastCompactionState() != null) { + foundCompactedSegments.add(segment); + } + } + Assert.assertEquals(foundCompactedSegments.size(), expectedCompactedSegmentCount); + for (DataSegment compactedSegment : foundCompactedSegments) { + Assert.assertNotNull(compactedSegment.getLastCompactionState()); + Assert.assertNotNull(compactedSegment.getLastCompactionState().getPartitionsSpec()); + Assert.assertEquals( + compactedSegment.getLastCompactionState().getPartitionsSpec().getType(), + SecondaryPartitionType.LINEAR + ); + } + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/AbstractIndexerTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/AbstractIndexerTest.java new file mode 100644 index 00000000000..5e53330e705 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/AbstractIndexerTest.java @@ -0,0 +1,188 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.indexer; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.inject.Inject; +import org.apache.commons.io.IOUtils; +import org.apache.druid.guice.annotations.Json; +import org.apache.druid.guice.annotations.Smile; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.testing.IntegrationTestingConfig; +import org.apache.druid.testing.clients.CoordinatorResourceTestClient; +import org.apache.druid.testing.clients.OverlordResourceTestClient; +import org.apache.druid.testing.clients.TaskResponseObject; +import org.apache.druid.testing.utils.ITRetryUtil; +import org.apache.druid.testing.utils.TestQueryHelper; +import org.joda.time.Interval; + +import java.io.BufferedReader; +import java.io.Closeable; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.Callable; + +public abstract class AbstractIndexerTest +{ + private static final Logger LOG = new Logger(AbstractIndexerTest.class); + + @Inject + protected CoordinatorResourceTestClient coordinator; + @Inject + protected OverlordResourceTestClient indexer; + @Inject + @Json + protected ObjectMapper jsonMapper; + @Inject + @Smile + protected ObjectMapper smileMapper; + @Inject + protected TestQueryHelper queryHelper; + + @Inject + protected IntegrationTestingConfig config; + + protected Closeable unloader(final String dataSource) + { + return () -> unloadAndKillData(dataSource); + } + + protected void unloadAndKillData(final String dataSource) + { + // Get all failed task logs + List allTasks = indexer.getCompleteTasksForDataSource(dataSource); + for (TaskResponseObject task : allTasks) { + if (task.getStatus().isFailure()) { + LOG.info("------- START Found failed task logging for taskId=" + task.getId() + " -------"); + LOG.info("Start failed task log:"); + LOG.info(indexer.getTaskLog(task.getId())); + LOG.info("End failed task log."); + LOG.info("Start failed task errorMsg:"); + LOG.info(indexer.getTaskErrorMessage(task.getId())); + LOG.info("End failed task errorMsg."); + LOG.info("------- END Found failed task logging for taskId=" + task.getId() + " -------"); + } + } + + List intervals = coordinator.getSegmentIntervals(dataSource); + + // each element in intervals has this form: + // 2015-12-01T23:15:00.000Z/2015-12-01T23:16:00.000Z + // we'll sort the list (ISO dates have lexicographic order) + // then delete segments from the 1st date in the first string + // to the 2nd date in the last string + Collections.sort(intervals); + String first = intervals.get(0).split("/")[0]; + String last = intervals.get(intervals.size() - 1).split("/")[1]; + unloadAndKillData(dataSource, first, last); + } + + protected String submitIndexTask(String indexTask, final String fullDatasourceName) throws Exception + { + String taskSpec = getResourceAsString(indexTask); + taskSpec = StringUtils.replace(taskSpec, "%%DATASOURCE%%", fullDatasourceName); + taskSpec = StringUtils.replace( + taskSpec, + "%%SEGMENT_AVAIL_TIMEOUT_MILLIS%%", + jsonMapper.writeValueAsString("0") + ); + final String taskID = indexer.submitTask(taskSpec); + LOG.info("TaskID for loading index task %s", taskID); + + return taskID; + } + + protected void loadData(String indexTask, final String fullDatasourceName) throws Exception + { + final String taskID = submitIndexTask(indexTask, fullDatasourceName); + indexer.waitUntilTaskCompletes(taskID); + + ITRetryUtil.retryUntilTrue( + () -> coordinator.areSegmentsLoaded(fullDatasourceName), + "Segment Load" + ); + } + + private void unloadAndKillData(final String dataSource, String start, String end) + { + // Wait for any existing index tasks to complete before disabling the datasource otherwise + // realtime tasks can get stuck waiting for handoff. https://github.com/apache/druid/issues/1729 + waitForAllTasksToCompleteForDataSource(dataSource); + Interval interval = Intervals.of(start + "/" + end); + coordinator.unloadSegmentsForDataSource(dataSource); + ITRetryUtil.retryUntilFalse( + new Callable() + { + @Override + public Boolean call() + { + return coordinator.areSegmentsLoaded(dataSource); + } + }, "Segment Unloading" + ); + coordinator.deleteSegmentsDataSource(dataSource, interval); + waitForAllTasksToCompleteForDataSource(dataSource); + } + + protected void waitForAllTasksToCompleteForDataSource(final String dataSource) + { + ITRetryUtil.retryUntilTrue( + () -> (indexer.getUncompletedTasksForDataSource(dataSource).size() == 0), + StringUtils.format("Waiting for all tasks of [%s] to complete", dataSource) + ); + } + + public static String getResourceAsString(String file) throws IOException + { + try (final InputStream inputStream = getResourceAsStream(file)) { + return IOUtils.toString(inputStream, StandardCharsets.UTF_8); + } + } + + public static InputStream getResourceAsStream(String resource) + { + return AbstractIndexerTest.class.getResourceAsStream(resource); + } + + public static List listResources(String dir) throws IOException + { + List resources = new ArrayList<>(); + + try ( + InputStream in = getResourceAsStream(dir); + BufferedReader br = new BufferedReader(new InputStreamReader(in, StringUtils.UTF8_STRING)) + ) { + String resource; + + while ((resource = br.readLine()) != null) { + resources.add(resource); + } + } + + return resources; + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/AbstractLocalInputSourceParallelIndexTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/AbstractLocalInputSourceParallelIndexTest.java new file mode 100644 index 00000000000..ce8a9f5c13c --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/AbstractLocalInputSourceParallelIndexTest.java @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.indexer; + +import com.google.common.collect.ImmutableMap; +import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.StringUtils; + +import javax.annotation.Nonnull; +import java.io.Closeable; +import java.util.Map; +import java.util.UUID; +import java.util.function.Function; + +public abstract class AbstractLocalInputSourceParallelIndexTest extends AbstractITBatchIndexTest +{ + private static final String INDEX_TASK = "/indexer/wikipedia_local_input_source_index_task.json"; + private static final String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_index_queries.json"; + + public void doIndexTest( + InputFormatDetails inputFormatDetails, + Pair segmentAvailabilityConfirmationPair + ) throws Exception + { + doIndexTest(inputFormatDetails, ImmutableMap.of(), segmentAvailabilityConfirmationPair); + } + + public void doIndexTest( + InputFormatDetails inputFormatDetails, + @Nonnull Map extraInputFormatMap, + Pair segmentAvailabilityConfirmationPair + ) throws Exception + { + final String indexDatasource = "wikipedia_index_test_" + UUID.randomUUID(); + Map inputFormatMap = new ImmutableMap.Builder().putAll(extraInputFormatMap) + .put("type", inputFormatDetails.getInputFormatType()) + .build(); + try ( + final Closeable ignored1 = unloader(indexDatasource + config.getExtraDatasourceNameSuffix()); + ) { + final Function sqlInputSourcePropsTransform = spec -> { + try { + spec = StringUtils.replace( + spec, + "%%PARTITIONS_SPEC%%", + jsonMapper.writeValueAsString(new DynamicPartitionsSpec(null, null)) + ); + spec = StringUtils.replace( + spec, + "%%INPUT_SOURCE_FILTER%%", + "*" + inputFormatDetails.getFileExtension() + ); + spec = StringUtils.replace( + spec, + "%%INPUT_SOURCE_BASE_DIR%%", + "/resources/data/batch_index" + inputFormatDetails.getFolderSuffix() + ); + spec = StringUtils.replace( + spec, + "%%INPUT_FORMAT%%", + jsonMapper.writeValueAsString(inputFormatMap) + ); + spec = StringUtils.replace( + spec, + "%%APPEND_TO_EXISTING%%", + jsonMapper.writeValueAsString(false) + ); + spec = StringUtils.replace( + spec, + "%%DROP_EXISTING%%", + jsonMapper.writeValueAsString(false) + ); + spec = StringUtils.replace( + spec, + "%%FORCE_GUARANTEED_ROLLUP%%", + jsonMapper.writeValueAsString(false) + ); + return spec; + } + catch (Exception e) { + throw new RuntimeException(e); + } + }; + + doIndexTest( + indexDatasource, + INDEX_TASK, + sqlInputSourcePropsTransform, + INDEX_QUERIES_RESOURCE, + false, + true, + true, + segmentAvailabilityConfirmationPair + ); + } + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITBestEffortRollupParallelIndexTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITBestEffortRollupParallelIndexTest.java new file mode 100644 index 00000000000..96e8d500fa6 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITBestEffortRollupParallelIndexTest.java @@ -0,0 +1,250 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.indexer; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.google.inject.Inject; +import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; +import org.apache.druid.indexer.partitions.PartitionsSpec; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; +import org.apache.druid.testing.clients.CoordinatorResourceTestClient; +import org.apache.druid.testing.utils.ITRetryUtil; +import org.apache.druid.testsEx.categories.BatchIndex; +import org.apache.druid.testsEx.config.DruidTestRunner; +import org.junit.Assert; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; + +import java.io.Closeable; +import java.util.function.Function; + +@RunWith(DruidTestRunner.class) +@Category(BatchIndex.class) +public class ITBestEffortRollupParallelIndexTest extends AbstractITBatchIndexTest +{ + // This ingestion spec has a splitHintSpec of maxSplitSize of 1 to test whether or not the task can handle + // maxSplitSize of 1 properly. + private static final String INDEX_TASK = "/indexer/wikipedia_parallel_index_task.json"; + private static final String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_parallel_index_queries.json"; + private static final String REINDEX_TASK = "/indexer/wikipedia_parallel_reindex_task.json"; + private static final String REINDEX_QUERIES_RESOURCE = "/indexer/wikipedia_parallel_reindex_queries.json"; + private static final String INDEX_DATASOURCE = "wikipedia_parallel_index_test"; + private static final String INDEX_INGEST_SEGMENT_DATASOURCE = "wikipedia_parallel_ingest_segment_index_test"; + private static final String INDEX_INGEST_SEGMENT_TASK = "/indexer/wikipedia_parallel_ingest_segment_index_task.json"; + private static final String INDEX_DRUID_INPUT_SOURCE_DATASOURCE = "wikipedia_parallel_druid_input_source_index_test"; + private static final String INDEX_DRUID_INPUT_SOURCE_TASK = "/indexer/wikipedia_parallel_druid_input_source_index_task.json"; + + private static final CoordinatorDynamicConfig DYNAMIC_CONFIG_PAUSED = + CoordinatorDynamicConfig.builder().withPauseCoordination(true).build(); + private static final CoordinatorDynamicConfig DYNAMIC_CONFIG_DEFAULT = + CoordinatorDynamicConfig.builder().build(); + + @Inject + CoordinatorResourceTestClient coordinatorClient; + + @Test + public void testIndexData() throws Exception + { + PartitionsSpec partitionsSpec = new DynamicPartitionsSpec(null, null); + try ( + final Closeable ignored1 = unloader(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()); + final Closeable ignored2 = unloader(INDEX_INGEST_SEGMENT_DATASOURCE + config.getExtraDatasourceNameSuffix()); + final Closeable ignored3 = unloader(INDEX_DRUID_INPUT_SOURCE_DATASOURCE + config.getExtraDatasourceNameSuffix()) + ) { + boolean forceGuaranteedRollup = partitionsSpec.isForceGuaranteedRollupCompatible(); + Assert.assertFalse("parititionSpec does not support best-effort rollup", forceGuaranteedRollup); + + final Function rollupTransform = spec -> { + try { + spec = StringUtils.replace( + spec, + "%%FORCE_GUARANTEED_ROLLUP%%", + Boolean.toString(false) + ); + spec = StringUtils.replace( + spec, + "%%SEGMENT_AVAIL_TIMEOUT_MILLIS%%", + jsonMapper.writeValueAsString("0") + ); + return StringUtils.replace( + spec, + "%%PARTITIONS_SPEC%%", + jsonMapper.writeValueAsString(partitionsSpec) + ); + } + catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + }; + + doIndexTest( + INDEX_DATASOURCE, + INDEX_TASK, + rollupTransform, + INDEX_QUERIES_RESOURCE, + false, + true, + true, + new Pair<>(false, false) + ); + + // Index again, this time only choosing the second data file, and without explicit intervals chosen. + // The second datafile covers both day segments, so this should replace them, as reflected in the queries. + doIndexTest( + INDEX_DATASOURCE, + REINDEX_TASK, + rollupTransform, + REINDEX_QUERIES_RESOURCE, + true, + true, + true, + new Pair<>(false, false) + ); + + doReindexTest( + INDEX_DATASOURCE, + INDEX_INGEST_SEGMENT_DATASOURCE, + rollupTransform, + INDEX_INGEST_SEGMENT_TASK, + REINDEX_QUERIES_RESOURCE, + new Pair<>(false, false) + ); + + // with DruidInputSource instead of IngestSegmentFirehose + doReindexTest( + INDEX_DATASOURCE, + INDEX_DRUID_INPUT_SOURCE_DATASOURCE, + rollupTransform, + INDEX_DRUID_INPUT_SOURCE_TASK, + REINDEX_QUERIES_RESOURCE, + new Pair<>(false, false) + ); + } + } + + /** + * Test a non zero value for awaitSegmentAvailabilityTimeoutMillis. This will confirm that the report for the task + * indicates segments were confirmed to be available on the cluster before finishing the ingestion job. + */ + @Test + public void testIndexDataVerifySegmentAvailability() throws Exception + { + PartitionsSpec partitionsSpec = new DynamicPartitionsSpec(null, null); + try ( + final Closeable ignored1 = unloader(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()); + ) { + boolean forceGuaranteedRollup = partitionsSpec.isForceGuaranteedRollupCompatible(); + Assert.assertFalse("parititionSpec does not support best-effort rollup", forceGuaranteedRollup); + + final Function rollupTransform = spec -> { + try { + spec = StringUtils.replace( + spec, + "%%FORCE_GUARANTEED_ROLLUP%%", + Boolean.toString(false) + ); + spec = StringUtils.replace( + spec, + "%%SEGMENT_AVAIL_TIMEOUT_MILLIS%%", + jsonMapper.writeValueAsString("600000") + ); + return StringUtils.replace( + spec, + "%%PARTITIONS_SPEC%%", + jsonMapper.writeValueAsString(partitionsSpec) + ); + } + catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + }; + + doIndexTest( + INDEX_DATASOURCE, + INDEX_TASK, + rollupTransform, + INDEX_QUERIES_RESOURCE, + false, + true, + true, + new Pair<>(true, true) + ); + } + } + + /** + * Test a non zero value for awaitSegmentAvailabilityTimeoutMillis. Setting the config value to 1 millis + * and pausing coordination to confirm that the task will still succeed even if the job was not able to confirm the + * segments were loaded by the time the timeout occurs. + */ + @Test + public void testIndexDataAwaitSegmentAvailabilityFailsButTaskSucceeds() throws Exception + { + PartitionsSpec partitionsSpec = new DynamicPartitionsSpec(null, null); + try ( + final Closeable ignored1 = unloader(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()); + ) { + coordinatorClient.postDynamicConfig(DYNAMIC_CONFIG_PAUSED); + boolean forceGuaranteedRollup = partitionsSpec.isForceGuaranteedRollupCompatible(); + Assert.assertFalse("parititionSpec does not support best-effort rollup", forceGuaranteedRollup); + + final Function rollupTransform = spec -> { + try { + spec = StringUtils.replace( + spec, + "%%FORCE_GUARANTEED_ROLLUP%%", + Boolean.toString(false) + ); + spec = StringUtils.replace( + spec, + "%%SEGMENT_AVAIL_TIMEOUT_MILLIS%%", + jsonMapper.writeValueAsString("1") + ); + return StringUtils.replace( + spec, + "%%PARTITIONS_SPEC%%", + jsonMapper.writeValueAsString(partitionsSpec) + ); + } + catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + }; + + doIndexTest( + INDEX_DATASOURCE, + INDEX_TASK, + rollupTransform, + INDEX_QUERIES_RESOURCE, + false, + false, + false, + new Pair<>(true, false) + ); + coordinatorClient.postDynamicConfig(DYNAMIC_CONFIG_DEFAULT); + ITRetryUtil.retryUntilTrue( + () -> coordinator.areSegmentsLoaded(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()), "Segment Load" + ); + } + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITCombiningFirehoseFactoryIndexTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITCombiningFirehoseFactoryIndexTest.java new file mode 100644 index 00000000000..8d394a0297e --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITCombiningFirehoseFactoryIndexTest.java @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.indexer; + +import com.fasterxml.jackson.core.JsonProcessingException; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.testsEx.categories.BatchIndex; +import org.apache.druid.testsEx.config.DruidTestRunner; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; + +import java.io.Closeable; +import java.util.function.Function; + +@RunWith(DruidTestRunner.class) +@Category(BatchIndex.class) +public class ITCombiningFirehoseFactoryIndexTest extends AbstractITBatchIndexTest +{ + private static final String INDEX_TASK = "/indexer/wikipedia_index_task.json"; + private static final String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_index_queries.json"; + private static final String INDEX_DATASOURCE = "wikipedia_index_test"; + + private static final String COMBINING_INDEX_TASK = "/indexer/wikipedia_combining_firehose_index_task.json"; + private static final String COMBINING_QUERIES_RESOURCE = "/indexer/wikipedia_combining_firehose_index_queries.json"; + private static final String COMBINING_INDEX_DATASOURCE = "wikipedia_comb_index_test"; + + @Test + public void testIndexData() throws Exception + { + try ( + final Closeable ignored1 = unloader(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()); + final Closeable ignored2 = unloader(COMBINING_INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()); + ) { + final Function combiningFirehoseSpecTransform = spec -> { + try { + return StringUtils.replace( + spec, + "%%COMBINING_DATASOURCE%%", + INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix() + ); + } + catch (Exception e) { + throw new RuntimeException(e); + } + }; + final Function transform = spec -> { + try { + return StringUtils.replace( + spec, + "%%SEGMENT_AVAIL_TIMEOUT_MILLIS%%", + jsonMapper.writeValueAsString("0") + ); + } + catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + }; + + doIndexTest( + INDEX_DATASOURCE, + INDEX_TASK, + transform, + INDEX_QUERIES_RESOURCE, + false, + true, + true, + new Pair<>(false, false) + ); + doIndexTest( + COMBINING_INDEX_DATASOURCE, + COMBINING_INDEX_TASK, + combiningFirehoseSpecTransform, + COMBINING_QUERIES_RESOURCE, + false, + true, + true, + new Pair<>(false, false) + ); + } + } + +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITCombiningInputSourceParallelIndexTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITCombiningInputSourceParallelIndexTest.java new file mode 100644 index 00000000000..5ec9ea0dbc4 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITCombiningInputSourceParallelIndexTest.java @@ -0,0 +1,130 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.indexer; + +import com.google.common.collect.ImmutableMap; +import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.testsEx.categories.BatchIndex; +import org.apache.druid.testsEx.config.DruidTestRunner; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; + +import java.io.Closeable; +import java.util.Map; +import java.util.function.Function; + +@RunWith(DruidTestRunner.class) +@Category(BatchIndex.class) +public class ITCombiningInputSourceParallelIndexTest extends AbstractITBatchIndexTest +{ + private static final String INDEX_TASK = "/indexer/wikipedia_local_input_source_index_task.json"; + private static final String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_index_queries.json"; + private static final String INDEX_DATASOURCE = "wikipedia_index_test"; + + private static final String COMBINING_INDEX_TASK = "/indexer/wikipedia_combining_input_source_index_parallel_task.json"; + private static final String COMBINING_QUERIES_RESOURCE = "/indexer/wikipedia_combining_firehose_index_queries.json"; + private static final String COMBINING_INDEX_DATASOURCE = "wikipedia_comb_index_test"; + + @Test + public void testIndexData() throws Exception + { + Map inputFormatMap = new ImmutableMap + .Builder() + .put("type", "json") + .build(); + try ( + final Closeable ignored1 = unloader(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()); + final Closeable ignored2 = unloader(COMBINING_INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()); + ) { + final Function combiningInputSourceSpecTransform = spec -> { + try { + spec = StringUtils.replace( + spec, + "%%PARTITIONS_SPEC%%", + jsonMapper.writeValueAsString(new DynamicPartitionsSpec(null, null)) + ); + spec = StringUtils.replace( + spec, + "%%INPUT_SOURCE_FILTER%%", + "wikipedia_index_data*" + ); + spec = StringUtils.replace( + spec, + "%%INPUT_SOURCE_BASE_DIR%%", + "/resources/data/batch_index/json" + ); + spec = StringUtils.replace( + spec, + "%%INPUT_FORMAT%%", + jsonMapper.writeValueAsString(inputFormatMap) + ); + spec = StringUtils.replace( + spec, + "%%APPEND_TO_EXISTING%%", + jsonMapper.writeValueAsString(false) + ); + spec = StringUtils.replace( + spec, + "%%DROP_EXISTING%%", + jsonMapper.writeValueAsString(false) + ); + spec = StringUtils.replace( + spec, + "%%FORCE_GUARANTEED_ROLLUP%%", + jsonMapper.writeValueAsString(false) + ); + spec = StringUtils.replace( + spec, + "%%COMBINING_DATASOURCE%%", + INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix() + ); + return spec; + } + catch (Exception e) { + throw new RuntimeException(e); + } + }; + + doIndexTest( + INDEX_DATASOURCE, + INDEX_TASK, + combiningInputSourceSpecTransform, + INDEX_QUERIES_RESOURCE, + false, + true, + true, + new Pair<>(false, false) + ); + doIndexTest( + COMBINING_INDEX_DATASOURCE, + COMBINING_INDEX_TASK, + combiningInputSourceSpecTransform, + COMBINING_QUERIES_RESOURCE, + false, + true, + true, + new Pair<>(false, false) + ); + } + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITIndexerTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITIndexerTest.java new file mode 100644 index 00000000000..65b8dc0b1ac --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITIndexerTest.java @@ -0,0 +1,386 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.indexer; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.google.inject.Inject; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; +import org.apache.druid.testing.clients.CoordinatorResourceTestClient; +import org.apache.druid.testing.utils.ITRetryUtil; +import org.apache.druid.testsEx.categories.BatchIndex; +import org.apache.druid.testsEx.config.DruidTestRunner; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; + +import java.io.Closeable; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.function.Function; + +@RunWith(DruidTestRunner.class) +@Category(BatchIndex.class) +public class ITIndexerTest extends AbstractITBatchIndexTest +{ + private static final String INDEX_TASK = "/indexer/wikipedia_index_task.json"; + private static final String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_index_queries.json"; + private static final String INDEX_DATASOURCE = "wikipedia_index_test"; + + private static final String INDEX_WITH_TIMESTAMP_TASK = "/indexer/wikipedia_with_timestamp_index_task.json"; + // TODO: add queries that validate timestamp is different from the __time column since it is a dimension + // TODO: https://github.com/apache/druid/issues/9565 + private static final String INDEX_WITH_TIMESTAMP_QUERIES_RESOURCE = "/indexer/wikipedia_index_queries.json"; + private static final String INDEX_WITH_TIMESTAMP_DATASOURCE = "wikipedia_with_timestamp_index_test"; + + private static final String REINDEX_TASK = "/indexer/wikipedia_reindex_task.json"; + private static final String REINDEX_TASK_WITH_DRUID_INPUT_SOURCE = "/indexer/wikipedia_reindex_druid_input_source_task.json"; + private static final String REINDEX_QUERIES_RESOURCE = "/indexer/wikipedia_reindex_queries.json"; + private static final String REINDEX_DATASOURCE = "wikipedia_reindex_test"; + + private static final String MERGE_INDEX_TASK = "/indexer/wikipedia_merge_index_task.json"; + private static final String MERGE_INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_merge_index_queries.json"; + private static final String MERGE_INDEX_DATASOURCE = "wikipedia_merge_index_test"; + + private static final String MERGE_REINDEX_TASK = "/indexer/wikipedia_merge_reindex_task.json"; + private static final String MERGE_REINDEX_TASK_WITH_DRUID_INPUT_SOURCE = "/indexer/wikipedia_merge_reindex_druid_input_source_task.json"; + private static final String MERGE_REINDEX_QUERIES_RESOURCE = "/indexer/wikipedia_merge_index_queries.json"; + private static final String MERGE_REINDEX_DATASOURCE = "wikipedia_merge_reindex_test"; + + private static final String INDEX_WITH_MERGE_COLUMN_LIMIT_TASK = "/indexer/wikipedia_index_with_merge_column_limit_task.json"; + private static final String INDEX_WITH_MERGE_COLUMN_LIMIT_DATASOURCE = "wikipedia_index_with_merge_column_limit_test"; + + private static final String GET_LOCKED_INTERVALS = "wikipedia_index_get_locked_intervals_test"; + + private static final CoordinatorDynamicConfig DYNAMIC_CONFIG_PAUSED = + CoordinatorDynamicConfig.builder().withPauseCoordination(true).build(); + private static final CoordinatorDynamicConfig DYNAMIC_CONFIG_DEFAULT = + CoordinatorDynamicConfig.builder().build(); + + @Inject + CoordinatorResourceTestClient coordinatorClient; + + @Test + public void testIndexData() throws Exception + { + final String reindexDatasource = REINDEX_DATASOURCE + "-testIndexData"; + final String reindexDatasourceWithDruidInputSource = REINDEX_DATASOURCE + "-testIndexData-druidInputSource"; + try ( + final Closeable ignored1 = unloader(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()); + final Closeable ignored2 = unloader(reindexDatasource + config.getExtraDatasourceNameSuffix()); + final Closeable ignored3 = unloader(reindexDatasourceWithDruidInputSource + config.getExtraDatasourceNameSuffix()) + ) { + + final Function transform = spec -> { + try { + return StringUtils.replace( + spec, + "%%SEGMENT_AVAIL_TIMEOUT_MILLIS%%", + jsonMapper.writeValueAsString("0") + ); + } + catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + }; + + doIndexTest( + INDEX_DATASOURCE, + INDEX_TASK, + transform, + INDEX_QUERIES_RESOURCE, + false, + true, + true, + new Pair<>(false, false) + ); + doReindexTest( + INDEX_DATASOURCE, + reindexDatasource, + REINDEX_TASK, + REINDEX_QUERIES_RESOURCE, + new Pair<>(false, false) + ); + doReindexTest( + INDEX_DATASOURCE, + reindexDatasourceWithDruidInputSource, + REINDEX_TASK_WITH_DRUID_INPUT_SOURCE, + REINDEX_QUERIES_RESOURCE, + new Pair<>(false, false) + ); + } + } + + @Test + public void testReIndexDataWithTimestamp() throws Exception + { + final String reindexDatasource = REINDEX_DATASOURCE + "-testReIndexDataWithTimestamp"; + final String reindexDatasourceWithDruidInputSource = REINDEX_DATASOURCE + "-testReIndexDataWithTimestamp-druidInputSource"; + try ( + final Closeable ignored1 = unloader(INDEX_WITH_TIMESTAMP_DATASOURCE + config.getExtraDatasourceNameSuffix()); + final Closeable ignored2 = unloader(reindexDatasource + config.getExtraDatasourceNameSuffix()); + final Closeable ignored3 = unloader(reindexDatasourceWithDruidInputSource + config.getExtraDatasourceNameSuffix()) + ) { + doIndexTest( + INDEX_WITH_TIMESTAMP_DATASOURCE, + INDEX_WITH_TIMESTAMP_TASK, + INDEX_WITH_TIMESTAMP_QUERIES_RESOURCE, + false, + true, + true, + new Pair<>(false, false) + ); + doReindexTest( + INDEX_WITH_TIMESTAMP_DATASOURCE, + reindexDatasource, + REINDEX_TASK, + REINDEX_QUERIES_RESOURCE, + new Pair<>(false, false) + ); + doReindexTest( + INDEX_WITH_TIMESTAMP_DATASOURCE, + reindexDatasourceWithDruidInputSource, + REINDEX_TASK_WITH_DRUID_INPUT_SOURCE, + REINDEX_QUERIES_RESOURCE, + new Pair<>(false, false) + ); + } + } + + @Test + public void testReIndexWithNonExistingDatasource() throws Exception + { + Pair dummyPair = new Pair<>(false, false); + final String fullBaseDatasourceName = "nonExistingDatasource2904"; + final String fullReindexDatasourceName = "newDatasource123"; + + String taskSpec = StringUtils.replace( + getResourceAsString(REINDEX_TASK_WITH_DRUID_INPUT_SOURCE), + "%%DATASOURCE%%", + fullBaseDatasourceName + ); + taskSpec = StringUtils.replace( + taskSpec, + "%%REINDEX_DATASOURCE%%", + fullReindexDatasourceName + ); + + // This method will also verify task is successful after task finish running + // We expect task to be successful even if the datasource to reindex does not exist + submitTaskAndWait( + taskSpec, + fullReindexDatasourceName, + false, + false, + dummyPair + ); + } + + @Test + public void testMERGEIndexData() throws Exception + { + final String reindexDatasource = MERGE_REINDEX_DATASOURCE + "-testMergeIndexData"; + final String reindexDatasourceWithDruidInputSource = MERGE_REINDEX_DATASOURCE + "-testMergeReIndexData-druidInputSource"; + try ( + final Closeable ignored1 = unloader(MERGE_INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()); + final Closeable ignored2 = unloader(reindexDatasource + config.getExtraDatasourceNameSuffix()); + final Closeable ignored3 = unloader(reindexDatasourceWithDruidInputSource + config.getExtraDatasourceNameSuffix()) + ) { + doIndexTest( + MERGE_INDEX_DATASOURCE, + MERGE_INDEX_TASK, + MERGE_INDEX_QUERIES_RESOURCE, + false, + true, + true, + new Pair<>(false, false) + ); + doReindexTest( + MERGE_INDEX_DATASOURCE, + reindexDatasource, + MERGE_REINDEX_TASK, + MERGE_REINDEX_QUERIES_RESOURCE, + new Pair<>(false, false) + ); + doReindexTest( + MERGE_INDEX_DATASOURCE, + reindexDatasourceWithDruidInputSource, + MERGE_REINDEX_TASK_WITH_DRUID_INPUT_SOURCE, + MERGE_INDEX_QUERIES_RESOURCE, + new Pair<>(false, false) + ); + } + } + + /** + * Test that task reports indicate the ingested segments were loaded before the configured timeout expired. + * + * @throws Exception + */ + @Test + public void testIndexDataAwaitSegmentAvailability() throws Exception + { + try ( + final Closeable ignored1 = unloader(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()); + ) { + final Function transform = spec -> { + try { + return StringUtils.replace( + spec, + "%%SEGMENT_AVAIL_TIMEOUT_MILLIS%%", + jsonMapper.writeValueAsString("600000") + ); + } + catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + }; + + doIndexTest( + INDEX_DATASOURCE, + INDEX_TASK, + transform, + INDEX_QUERIES_RESOURCE, + false, + true, + true, + new Pair<>(true, true) + ); + } + } + + /** + * Test that the task still succeeds if the segments do not become available before the configured wait timeout + * expires. + * + * @throws Exception + */ + @Test + public void testIndexDataAwaitSegmentAvailabilityFailsButTaskSucceeds() throws Exception + { + try ( + final Closeable ignored1 = unloader(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()); + ) { + coordinatorClient.postDynamicConfig(DYNAMIC_CONFIG_PAUSED); + final Function transform = spec -> { + try { + return StringUtils.replace( + spec, + "%%SEGMENT_AVAIL_TIMEOUT_MILLIS%%", + jsonMapper.writeValueAsString("1") + ); + } + catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + }; + + doIndexTest( + INDEX_DATASOURCE, + INDEX_TASK, + transform, + INDEX_QUERIES_RESOURCE, + false, + false, + false, + new Pair<>(true, false) + ); + coordinatorClient.postDynamicConfig(DYNAMIC_CONFIG_DEFAULT); + ITRetryUtil.retryUntilTrue( + () -> coordinator.areSegmentsLoaded(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()), "Segment Load" + ); + } + } + + + @Test + public void testIndexWithMergeColumnLimitData() throws Exception + { + try ( + final Closeable ignored1 = unloader(INDEX_WITH_MERGE_COLUMN_LIMIT_DATASOURCE + config.getExtraDatasourceNameSuffix()); + ) { + doIndexTest( + INDEX_WITH_MERGE_COLUMN_LIMIT_DATASOURCE, + INDEX_WITH_MERGE_COLUMN_LIMIT_TASK, + INDEX_QUERIES_RESOURCE, + false, + true, + true, + new Pair<>(false, false) + ); + } + } + + @Test + public void testGetLockedIntervals() throws Exception + { + final String datasourceName = GET_LOCKED_INTERVALS + config.getExtraDatasourceNameSuffix(); + try (final Closeable ignored = unloader(datasourceName)) { + // Submit an Indexing Task + submitIndexTask(INDEX_TASK, datasourceName); + + // Wait until it acquires a lock + final Map minTaskPriority = Collections.singletonMap(datasourceName, 0); + final Map> lockedIntervals = new HashMap<>(); + ITRetryUtil.retryUntilFalse( + () -> { + lockedIntervals.clear(); + lockedIntervals.putAll(indexer.getLockedIntervals(minTaskPriority)); + return lockedIntervals.isEmpty(); + }, + "Verify Intervals are Locked" + ); + + // Verify the locked intervals for this datasource + Assert.assertEquals(lockedIntervals.size(), 1); + Assert.assertEquals( + lockedIntervals.get(datasourceName), + Collections.singletonList(Intervals.of("2013-08-31/2013-09-02")) + ); + + ITRetryUtil.retryUntilTrue( + () -> coordinator.areSegmentsLoaded(datasourceName), + "Segment Load" + ); + } + } + + @Test + public void testJsonFunctions() throws Exception + { + final String taskSpec = getResourceAsString("/indexer/json_path_index_task.json"); + + submitTaskAndWait( + taskSpec, + "json_path_index_test", + false, + true, + new Pair<>(false, false) + ); + + doTestQuery("json_path_index_test", "/indexer/json_path_index_queries.json"); + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITOverwriteBatchIndexTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITOverwriteBatchIndexTest.java new file mode 100644 index 00000000000..73fc73d42d8 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITOverwriteBatchIndexTest.java @@ -0,0 +1,166 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.indexer; + +import com.google.common.collect.ImmutableMap; +import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.testsEx.categories.BatchIndex; +import org.apache.druid.testsEx.config.DruidTestRunner; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; + +import java.io.Closeable; +import java.util.Map; +import java.util.UUID; +import java.util.function.Function; + +@RunWith(DruidTestRunner.class) +@Category(BatchIndex.class) +public class ITOverwriteBatchIndexTest extends AbstractITBatchIndexTest +{ + private static final String INDEX_TASK = "/indexer/wikipedia_local_input_source_index_task.json"; + private static final String INDEX_QUERIES_ALL_INGESTION_RESOURCE = "/indexer/wikipedia_index_queries.json"; + private static final String INDEX_QUERIES_WITH_DROP_INGESTION_RESOURCE = "/indexer/wikipedia_index_queries_only_data3.json"; + private static final InputFormatDetails INPUT_FORMAT_DETAILS = InputFormatDetails.JSON; + private static final String ALL_DATA = "*" + INPUT_FORMAT_DETAILS.getFileExtension(); + private static final String ONE_DAY_DATA = "wikipedia_index_data3" + INPUT_FORMAT_DETAILS.getFileExtension(); + + @Test + public void doIndexTestWithOverwriteAndDrop() throws Exception + { + final String indexDatasource = "wikipedia_index_test_" + UUID.randomUUID(); + try ( + final Closeable ignored1 = unloader(indexDatasource + config.getExtraDatasourceNameSuffix()) + ) { + // Submit initial ingestion task + // The data interval is 2013-08-31 to 2013-09-02 with DAY segmentGranularity + // dropExisting true or false does not matter as there is no existing segments + // Even though there are only 10 rows in the three input files and that maxRowsPerSegment is the default + // there will be four segments generated due to the distribution of rows in the input files and the + // fact that there are three sub-tasks (two of the tasks generate one segment each and the other + // task generates two segments) + submitIngestionTaskAndVerify(indexDatasource, ALL_DATA, true); + verifySegmentsCountAndLoaded(indexDatasource, 4, 0); + doTestQuery(indexDatasource, INDEX_QUERIES_ALL_INGESTION_RESOURCE); + // Submit overwrite ingestion task with drop existing + // The ingestion task interval is the same as the first ingestion ("2013-08-31/2013-09-02"), + // however, the input data only contains one file with one day of data, 2013-09-02 + // with dropExisting flag set to true, after the second ingestion, we should expect to + // have two segments one for the sub-task that ingested the single data file containing one day + // and one segment for the tombstone to cover the other segment previously generated by the first ingestion + submitIngestionTaskAndVerify(indexDatasource, ONE_DAY_DATA, true); + verifySegmentsCountAndLoaded(indexDatasource, 2, 1); + doTestQuery(indexDatasource, INDEX_QUERIES_WITH_DROP_INGESTION_RESOURCE); + } + } + + @Test + public void doIndexTestWithOverwriteAndNoDrop() throws Exception + { + final String indexDatasource = "wikipedia_index_test_" + UUID.randomUUID(); + try ( + final Closeable ignored1 = unloader(indexDatasource + config.getExtraDatasourceNameSuffix()) + ) { + // Submit initial ingestion task + // The data interval is 2013-08-31 to 2013-09-02 with DAY segmentGranularity + // dropExisting true or false does not matter as there is no existing segments + submitIngestionTaskAndVerify(indexDatasource, ALL_DATA, false); + verifySegmentsCountAndLoaded(indexDatasource, 4, 0); + doTestQuery(indexDatasource, INDEX_QUERIES_ALL_INGESTION_RESOURCE); + // Submit overwrite ingestion task without drop existing + // The ingestion task interval is the same as the first ingestion ("2013-08-31/2013-09-02"), + // however, the input data only contains one day of data, 2013-09-02 (instead of 2 days of data) + // with dropExisting flag set to false, after the second ingestion, we should expect to have + // data from 2013-08-31/2013-09-01 remains unchanged and data for 2013-09-01/2013-09-02 from + // the second overwrite ingestion task + submitIngestionTaskAndVerify(indexDatasource, ONE_DAY_DATA, false); + verifySegmentsCountAndLoaded(indexDatasource, 3, 0); + doTestQuery(indexDatasource, INDEX_QUERIES_ALL_INGESTION_RESOURCE); + } + } + + private void submitIngestionTaskAndVerify( + String indexDatasource, + String fileFilter, + boolean dropExisting + ) throws Exception + { + Map inputFormatMap = new ImmutableMap.Builder() + .put("type", INPUT_FORMAT_DETAILS.getInputFormatType()) + .build(); + final Function sqlInputSourcePropsTransform = spec -> { + try { + spec = StringUtils.replace( + spec, + "%%PARTITIONS_SPEC%%", + jsonMapper.writeValueAsString(new DynamicPartitionsSpec(null, null)) + ); + spec = StringUtils.replace( + spec, + "%%INPUT_SOURCE_FILTER%%", + fileFilter + ); + spec = StringUtils.replace( + spec, + "%%INPUT_SOURCE_BASE_DIR%%", + "/resources/data/batch_index" + INPUT_FORMAT_DETAILS.getFolderSuffix() + ); + spec = StringUtils.replace( + spec, + "%%INPUT_FORMAT%%", + jsonMapper.writeValueAsString(inputFormatMap) + ); + spec = StringUtils.replace( + spec, + "%%APPEND_TO_EXISTING%%", + jsonMapper.writeValueAsString(false) + ); + spec = StringUtils.replace( + spec, + "%%DROP_EXISTING%%", + jsonMapper.writeValueAsString(dropExisting) + ); + spec = StringUtils.replace( + spec, + "%%FORCE_GUARANTEED_ROLLUP%%", + jsonMapper.writeValueAsString(false) + ); + return spec; + } + catch (Exception e) { + throw new RuntimeException(e); + } + }; + + doIndexTest( + indexDatasource, + INDEX_TASK, + sqlInputSourcePropsTransform, + null, + false, + false, + true, + new Pair<>(false, false) + ); + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITSystemTableBatchIndexTaskTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITSystemTableBatchIndexTaskTest.java new file mode 100644 index 00000000000..a5e4edc9580 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITSystemTableBatchIndexTaskTest.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.indexer; + +import com.fasterxml.jackson.core.JsonProcessingException; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.testsEx.categories.BatchIndex; +import org.apache.druid.testsEx.config.DruidTestRunner; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; + +import java.io.Closeable; +import java.util.function.Function; + +@RunWith(DruidTestRunner.class) +@Category(BatchIndex.class) +public class ITSystemTableBatchIndexTaskTest extends AbstractITBatchIndexTest +{ + private static final Logger LOG = new Logger(ITSystemTableBatchIndexTaskTest.class); + private static final String INDEX_TASK = "/indexer/wikipedia_index_task.json"; + private static final String SYSTEM_QUERIES_RESOURCE = "/indexer/sys_segment_batch_index_queries.json"; + private static final String INDEX_DATASOURCE = "wikipedia_index_test"; + + @Test + public void testIndexData() throws Exception + { + LOG.info("Starting batch index sys table queries"); + try ( + final Closeable ignored = unloader(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()) + ) { + + final Function transform = spec -> { + try { + return StringUtils.replace( + spec, + "%%SEGMENT_AVAIL_TIMEOUT_MILLIS%%", + jsonMapper.writeValueAsString("0") + ); + } + catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + }; + + doIndexTestSqlTest( + INDEX_DATASOURCE, + INDEX_TASK, + SYSTEM_QUERIES_RESOURCE, + transform + ); + } + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITTransformTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITTransformTest.java new file mode 100644 index 00000000000..61cb33fd2c6 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITTransformTest.java @@ -0,0 +1,125 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.indexer; + +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.testsEx.categories.BatchIndex; +import org.apache.druid.testsEx.config.DruidTestRunner; +import org.junit.Ignore; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; + +import java.io.Closeable; +import java.io.IOException; + +@RunWith(DruidTestRunner.class) +@Category(BatchIndex.class) +public class ITTransformTest extends AbstractITBatchIndexTest +{ + private static final String INDEX_TASK_WITH_FIREHOSE = "/indexer/wikipedia_index_task_with_transform.json"; + private static final String INDEX_TASK_WITH_INPUT_SOURCE = "/indexer/wikipedia_index_task_with_inputsource_transform.json"; + private static final String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_index_queries_with_transform.json"; + private static final String INDEX_DATASOURCE = "wikipedia_index_test"; + + private static final String REINDEX_TASK = "/indexer/wikipedia_reindex_task_with_transforms.json"; + private static final String REINDEX_TASK_WITH_DRUID_INPUT_SOURCE = "/indexer/wikipedia_reindex_druid_input_source_task_with_transforms.json"; + private static final String REINDEX_QUERIES_RESOURCE = "/indexer/wikipedia_reindex_queries_with_transforms.json"; + private static final String REINDEX_DATASOURCE = "wikipedia_reindex_test"; + + @Test + public void testIndexAndReIndexWithTransformSpec() throws IOException + { + final String reindexDatasourceWithDruidInputSource = REINDEX_DATASOURCE + "-druidInputSource"; + + try ( + final Closeable ignored1 = unloader(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()); + final Closeable ignored2 = unloader(reindexDatasourceWithDruidInputSource + config.getExtraDatasourceNameSuffix()) + ) { + doIndexTest( + INDEX_DATASOURCE, + INDEX_TASK_WITH_INPUT_SOURCE, + INDEX_QUERIES_RESOURCE, + false, + true, + true, + new Pair<>(false, false) + ); + doReindexTest( + INDEX_DATASOURCE, + reindexDatasourceWithDruidInputSource, + REINDEX_TASK_WITH_DRUID_INPUT_SOURCE, + REINDEX_QUERIES_RESOURCE, + new Pair<>(false, false) + ); + } + } + + @Test + @Ignore + public void testIndexAndReIndexUsingIngestSegmentWithTransforms() throws IOException + { + // TODO: re-instate this test when https://github.com/apache/druid/issues/9591 is fixed + // Move the re-index step into testIndexAndReIndexWithTransformSpec for faster tests! + final String reindexDatasource = REINDEX_DATASOURCE + "-testIndexData"; + try ( + final Closeable ignored1 = unloader(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()); + final Closeable ignored2 = unloader(reindexDatasource + config.getExtraDatasourceNameSuffix()) + ) { + doIndexTest( + INDEX_DATASOURCE, + INDEX_TASK_WITH_INPUT_SOURCE, + INDEX_QUERIES_RESOURCE, + false, + true, + true, + new Pair<>(false, false) + ); + doReindexTest( + INDEX_DATASOURCE, + reindexDatasource, + REINDEX_TASK, + REINDEX_QUERIES_RESOURCE, + new Pair<>(false, false) + ); + } + } + + @Test + @Ignore + public void testIndexWithFirehoseAndTransforms() throws IOException + { + // TODO: re-instate this test when https://github.com/apache/druid/issues/9589 is fixed + final String indexDatasource = INDEX_DATASOURCE + "-firehose"; + try ( + final Closeable ignored1 = unloader(indexDatasource + config.getExtraDatasourceNameSuffix()); + ) { + doIndexTest( + indexDatasource, + INDEX_TASK_WITH_FIREHOSE, + INDEX_QUERIES_RESOURCE, + false, + true, + true, + new Pair<>(false, false) + ); + } + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/leadership/ITHighAvailabilityTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/leadership/ITHighAvailabilityTest.java new file mode 100644 index 00000000000..735bc0a50ff --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/leadership/ITHighAvailabilityTest.java @@ -0,0 +1,246 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.leadership; + +import com.google.inject.Inject; +import org.apache.druid.cli.CliCustomNodeRole; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.discovery.DiscoveryDruidNode; +import org.apache.druid.discovery.DruidNodeDiscovery; +import org.apache.druid.discovery.DruidNodeDiscoveryProvider; +import org.apache.druid.discovery.NodeRole; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.java.util.http.client.HttpClient; +import org.apache.druid.testing.IntegrationTestingConfig; +import org.apache.druid.testing.guice.TestClient; +import org.apache.druid.testing.utils.SqlTestQueryHelper; +import org.apache.druid.tests.indexer.AbstractIndexerTest; +import org.apache.druid.testsEx.categories.HighAvailability; +import org.apache.druid.testsEx.cluster.DruidClusterClient; +import org.apache.druid.testsEx.config.DruidTestRunner; +import org.apache.druid.testsEx.config.Initializer; +import org.apache.druid.testsEx.utils.DruidClusterAdminClient; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; + +import java.util.List; +import java.util.Map; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +@RunWith(DruidTestRunner.class) +@Category(HighAvailability.class) +public class ITHighAvailabilityTest +{ + private static final Logger LOG = new Logger(ITHighAvailabilityTest.class); + private static final String SYSTEM_QUERIES_RESOURCE = Initializer.queryFile(HighAvailability.class, "sys.json"); + private static final int NUM_LEADERSHIP_SWAPS = 3; + + @Inject + private IntegrationTestingConfig config; + + @Inject + private DruidClusterAdminClient druidClusterAdminClient; + + @Inject + private DruidNodeDiscoveryProvider druidNodeDiscovery; + + @Inject + private SqlTestQueryHelper queryHelper; + + @Inject + @TestClient + private HttpClient httpClient; + + @Inject + private DruidClusterClient clusterClient; + + @Test + public void testLeadershipChanges() throws Exception + { + int runCount = 0; + String previousCoordinatorLeader = null; + String previousOverlordLeader = null; + // fetch current leaders, make sure queries work, then swap leaders and do it again + do { + String coordinatorLeader = getLeader("coordinator"); + String overlordLeader = getLeader("indexer"); + + // we expect leadership swap to happen + assertNotEquals(previousCoordinatorLeader, coordinatorLeader); + assertNotEquals(previousOverlordLeader, overlordLeader); + + previousCoordinatorLeader = coordinatorLeader; + previousOverlordLeader = overlordLeader; + + String queries = fillTemplate( + AbstractIndexerTest.getResourceAsString(SYSTEM_QUERIES_RESOURCE), + overlordLeader, + coordinatorLeader + ); + queryHelper.testQueriesFromString(queries); + + swapLeadersAndWait(coordinatorLeader, overlordLeader); + } while (runCount++ < NUM_LEADERSHIP_SWAPS); + } + + @Test + public void testDiscoveryAndSelfDiscovery() + { + // The cluster used here has an abbreviated set of services. + verifyRoleDiscovery(NodeRole.BROKER, 1); + verifyRoleDiscovery(NodeRole.COORDINATOR, 2); + verifyRoleDiscovery(NodeRole.OVERLORD, 2); + verifyRoleDiscovery(NodeRole.ROUTER, 1); + } + + public void verifyRoleDiscovery(NodeRole role, int expectedCount) + { + DruidNodeDiscovery discovered = druidNodeDiscovery.getForNodeRole(role); + try { + int count = 0; + for (DiscoveryDruidNode node : discovered.getAllNodes()) { + if (clusterClient.selfDiscovered(clusterClient.nodeUrl(node.getDruidNode()))) { + count++; + } + } + assertEquals(expectedCount, count); + } + catch (Exception e) { + LOG.error(e, "node discovery failed"); + fail(); + } + } + + @Test + public void testCustomDiscovery() + { + verifyRoleDiscovery(CliCustomNodeRole.NODE_ROLE, 1); + verifyCoordinatorCluster(); + } + + private void swapLeadersAndWait(String coordinatorLeader, String overlordLeader) + { + String coordUrl; + String coordLabel; + if (isCoordinatorOneLeader(coordinatorLeader)) { + druidClusterAdminClient.restartCoordinatorContainer(); + coordUrl = config.getCoordinatorUrl(); + coordLabel = "coordinator one"; + } else { + druidClusterAdminClient.restartCoordinatorTwoContainer(); + coordUrl = config.getCoordinatorTwoUrl(); + coordLabel = "coordinator two"; + } + + String overlordUrl; + String overlordLabel; + if (isOverlordOneLeader(overlordLeader)) { + druidClusterAdminClient.restartOverlordContainer(); + overlordUrl = config.getOverlordUrl(); + overlordLabel = "overlord one"; + } else { + druidClusterAdminClient.restartOverlordTwoContainer(); + overlordUrl = config.getOverlordTwoUrl(); + overlordLabel = "overlord two"; + } + clusterClient.waitForNodeReady(coordLabel, coordUrl); + clusterClient.waitForNodeReady(overlordLabel, overlordUrl); + } + + private String getLeader(String service) + { + return clusterClient.getLeader(service); + } + + private String fillTemplate(String template, String overlordLeader, String coordinatorLeader) + { + /* + {"host":"%%BROKER%%","server_type":"broker", "is_leader": %%NON_LEADER%%}, + {"host":"%%COORDINATOR_ONE%%","server_type":"coordinator", "is_leader": %%COORDINATOR_ONE_LEADER%%}, + {"host":"%%COORDINATOR_TWO%%","server_type":"coordinator", "is_leader": %%COORDINATOR_TWO_LEADER%%}, + {"host":"%%OVERLORD_ONE%%","server_type":"overlord", "is_leader": %%OVERLORD_ONE_LEADER%%}, + {"host":"%%OVERLORD_TWO%%","server_type":"overlord", "is_leader": %%OVERLORD_TWO_LEADER%%}, + {"host":"%%ROUTER%%","server_type":"router", "is_leader": %%NON_LEADER%%}, + */ + String working = template; + working = StringUtils.replace(working, "%%OVERLORD_ONE%%", config.getOverlordInternalHost()); + working = StringUtils.replace(working, "%%OVERLORD_TWO%%", config.getOverlordTwoInternalHost()); + working = StringUtils.replace(working, "%%COORDINATOR_ONE%%", config.getCoordinatorInternalHost()); + working = StringUtils.replace(working, "%%COORDINATOR_TWO%%", config.getCoordinatorTwoInternalHost()); + working = StringUtils.replace(working, "%%BROKER%%", config.getBrokerInternalHost()); + working = StringUtils.replace(working, "%%ROUTER%%", config.getRouterInternalHost()); + if (isOverlordOneLeader(overlordLeader)) { + working = StringUtils.replace(working, "%%OVERLORD_ONE_LEADER%%", "1"); + working = StringUtils.replace(working, "%%OVERLORD_TWO_LEADER%%", "0"); + } else { + working = StringUtils.replace(working, "%%OVERLORD_ONE_LEADER%%", "0"); + working = StringUtils.replace(working, "%%OVERLORD_TWO_LEADER%%", "1"); + } + if (isCoordinatorOneLeader(coordinatorLeader)) { + working = StringUtils.replace(working, "%%COORDINATOR_ONE_LEADER%%", "1"); + working = StringUtils.replace(working, "%%COORDINATOR_TWO_LEADER%%", "0"); + } else { + working = StringUtils.replace(working, "%%COORDINATOR_ONE_LEADER%%", "0"); + working = StringUtils.replace(working, "%%COORDINATOR_TWO_LEADER%%", "1"); + } + working = StringUtils.replace(working, "%%NON_LEADER%%", String.valueOf(NullHandling.defaultLongValue())); + return working; + } + + private boolean isCoordinatorOneLeader(String coordinatorLeader) + { + return coordinatorLeader.contains(transformHost(config.getCoordinatorInternalHost())); + } + + private boolean isOverlordOneLeader(String overlordLeader) + { + return overlordLeader.contains(transformHost(config.getOverlordInternalHost())); + } + + /** + * host + ':' which should be enough to distinguish subsets, e.g. 'druid-coordinator:8081' from + * 'druid-coordinator-two:8081' for example + */ + private static String transformHost(String host) + { + return StringUtils.format("%s:", host); + } + + private void verifyCoordinatorCluster() + { + // Verify the basics: 4 service types, excluding the custom node role. + // One of the two-node services has a size of 2. + // This endpoint includes an entry for historicals, even if none are running. + Map results = clusterClient.coordinatorCluster(); + assertEquals(5, results.size()); + @SuppressWarnings("unchecked") + List coordNodes = (List) results.get(NodeRole.COORDINATOR.getJsonName()); + assertEquals(2, coordNodes.size()); + @SuppressWarnings("unchecked") + List histNodes = (List) results.get(NodeRole.HISTORICAL.getJsonName()); + assertTrue(histNodes.isEmpty()); + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/utils/DruidClusterAdminClient.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/utils/DruidClusterAdminClient.java new file mode 100644 index 00000000000..60bd64ef847 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/utils/DruidClusterAdminClient.java @@ -0,0 +1,328 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.utils; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.github.dockerjava.api.DockerClient; +import com.github.dockerjava.api.command.ExecCreateCmdResponse; +import com.github.dockerjava.api.model.Container; +import com.github.dockerjava.core.DockerClientBuilder; +import com.github.dockerjava.core.command.ExecStartResultCallback; +import com.github.dockerjava.netty.NettyDockerCmdExecFactory; +import com.google.inject.Inject; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.java.util.http.client.HttpClient; +import org.apache.druid.java.util.http.client.Request; +import org.apache.druid.java.util.http.client.response.StatusResponseHandler; +import org.apache.druid.java.util.http.client.response.StatusResponseHolder; +import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; +import org.apache.druid.testing.IntegrationTestingConfig; +import org.apache.druid.testing.guice.TestClient; +import org.apache.druid.testing.utils.ITRetryUtil; +import org.jboss.netty.channel.ChannelException; +import org.jboss.netty.handler.codec.http.HttpMethod; +import org.jboss.netty.handler.codec.http.HttpResponseStatus; + +import java.io.ByteArrayOutputStream; +import java.net.URL; +import java.nio.channels.ClosedChannelException; +import java.nio.charset.StandardCharsets; +import java.util.Arrays; +import java.util.List; +import java.util.Optional; + +public class DruidClusterAdminClient +{ + public static final String COORDINATOR_DOCKER_CONTAINER_NAME = "/coordinator-one"; + public static final String COORDINATOR_TWO_DOCKER_CONTAINER_NAME = "/coordinator-two"; + public static final String HISTORICAL_DOCKER_CONTAINER_NAME = "/historical"; + public static final String OVERLORD_DOCKER_CONTAINER_NAME = "/overlord-one"; + public static final String OVERLORD_TWO_DOCKER_CONTAINER_NAME = "/overlord-two"; + public static final String BROKER_DOCKER_CONTAINER_NAME = "/broker"; + public static final String ROUTER_DOCKER_CONTAINER_NAME = "/router"; + public static final String MIDDLEMANAGER_DOCKER_CONTAINER_NAME = "/middlemanager"; + + private static final Logger LOG = new Logger(DruidClusterAdminClient.class); + + private final ObjectMapper jsonMapper; + private final HttpClient httpClient; + private IntegrationTestingConfig config; + + @Inject + DruidClusterAdminClient( + ObjectMapper jsonMapper, + @TestClient HttpClient httpClient, + IntegrationTestingConfig config + ) + { + this.jsonMapper = jsonMapper; + this.httpClient = httpClient; + this.config = config; + } + + public void restartCoordinatorContainer() + { + restartDockerContainer(COORDINATOR_DOCKER_CONTAINER_NAME); + } + + public void restartCoordinatorTwoContainer() + { + restartDockerContainer(COORDINATOR_TWO_DOCKER_CONTAINER_NAME); + } + + public void restartHistoricalContainer() + { + restartDockerContainer(HISTORICAL_DOCKER_CONTAINER_NAME); + } + + public void restartOverlordContainer() + { + restartDockerContainer(OVERLORD_DOCKER_CONTAINER_NAME); + } + + public void restartOverlordTwoContainer() + { + restartDockerContainer(OVERLORD_TWO_DOCKER_CONTAINER_NAME); + } + + public void restartBrokerContainer() + { + restartDockerContainer(BROKER_DOCKER_CONTAINER_NAME); + } + + public void restartRouterContainer() + { + restartDockerContainer(ROUTER_DOCKER_CONTAINER_NAME); + } + + public void restartMiddleManagerContainer() + { + restartDockerContainer(MIDDLEMANAGER_DOCKER_CONTAINER_NAME); + } + + public void waitUntilCoordinatorReady() + { + waitUntilInstanceReady(config.getCoordinatorUrl()); + postDynamicConfig(CoordinatorDynamicConfig.builder() + .withLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments(1) + .build()); + } + + public void waitUntilCoordinatorTwoReady() + { + waitUntilInstanceReady(config.getCoordinatorTwoUrl()); + postDynamicConfig(CoordinatorDynamicConfig.builder() + .withLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments(1) + .build()); + } + + public void waitUntilOverlordTwoReady() + { + waitUntilInstanceReady(config.getOverlordTwoUrl()); + } + + public void waitUntilHistoricalReady() + { + waitUntilInstanceReady(config.getHistoricalUrl()); + } + + public void waitUntilIndexerReady() + { + waitUntilInstanceReady(config.getOverlordUrl()); + } + + public void waitUntilBrokerReady() + { + waitUntilInstanceReady(config.getBrokerUrl()); + } + + public void waitUntilRouterReady() + { + waitUntilInstanceReady(config.getRouterUrl()); + } + + public Pair runCommandInCoordinatorContainer(String... cmd) throws Exception + { + return runCommandInDockerContainer(COORDINATOR_DOCKER_CONTAINER_NAME, cmd); + } + + public Pair runCommandInCoordinatorTwoContainer(String... cmd) throws Exception + { + return runCommandInDockerContainer(COORDINATOR_TWO_DOCKER_CONTAINER_NAME, cmd); + } + + public Pair runCommandInHistoricalContainer(String... cmd) throws Exception + { + return runCommandInDockerContainer(HISTORICAL_DOCKER_CONTAINER_NAME, cmd); + } + + public Pair runCommandInOverlordContainer(String... cmd) throws Exception + { + return runCommandInDockerContainer(OVERLORD_DOCKER_CONTAINER_NAME, cmd); + } + + public Pair runCommandInOverlordTwoContainer(String... cmd) throws Exception + { + return runCommandInDockerContainer(OVERLORD_TWO_DOCKER_CONTAINER_NAME, cmd); + } + + public Pair runCommandInBrokerContainer(String... cmd) throws Exception + { + return runCommandInDockerContainer(BROKER_DOCKER_CONTAINER_NAME, cmd); + } + + public Pair runCommandInRouterContainer(String... cmd) throws Exception + { + return runCommandInDockerContainer(ROUTER_DOCKER_CONTAINER_NAME, cmd); + } + + public Pair runCommandInMiddleManagerContainer(String... cmd) throws Exception + { + return runCommandInDockerContainer(MIDDLEMANAGER_DOCKER_CONTAINER_NAME, cmd); + } + + public Pair runCommandInDockerContainer(String serviceName, String... cmd) throws Exception + { + DockerClient dockerClient = newClient(); + ByteArrayOutputStream stdout = new ByteArrayOutputStream(); + ByteArrayOutputStream stderr = new ByteArrayOutputStream(); + ExecCreateCmdResponse execCreateCmdResponse = dockerClient.execCreateCmd(findDockerContainer(dockerClient, serviceName)) + .withAttachStderr(true) + .withAttachStdout(true) + .withCmd(cmd) + .exec(); + dockerClient.execStartCmd(execCreateCmdResponse.getId()) + .exec(new ExecStartResultCallback(stdout, stderr)) + .awaitCompletion(); + + return new Pair<>(stdout.toString(StandardCharsets.UTF_8.name()), stderr.toString(StandardCharsets.UTF_8.name())); + } + + public void restartDockerContainer(String serviceName) + { + DockerClient dockerClient = newClient(); + dockerClient.restartContainerCmd(findDockerContainer(dockerClient, serviceName)).exec(); + } + + public void killAndRestartDockerContainer(String serviceName) + { + final DockerClient dockerClient = newClient(); + final String containerId = findDockerContainer(dockerClient, serviceName); + + dockerClient.killContainerCmd(containerId).withSignal("SIGKILL").exec(); + dockerClient.startContainerCmd(containerId).exec(); + } + + private static DockerClient newClient() + { + return DockerClientBuilder + .getInstance() + .withDockerCmdExecFactory((new NettyDockerCmdExecFactory()).withConnectTimeout(10 * 1000)) + .build(); + } + + private String findDockerContainer(DockerClient dockerClient, String serviceName) + { + + List containers = dockerClient.listContainersCmd().exec(); + Optional containerName = containers + .stream() + .filter(container -> Arrays.asList(container.getNames()).contains(serviceName)) + .findFirst() + .map(Container::getId); + + if (!containerName.isPresent()) { + LOG.error("Cannot find docker container for " + serviceName); + throw new ISE("Cannot find docker container for " + serviceName); + } + return containerName.get(); + } + + private void waitUntilInstanceReady(final String host) + { + ITRetryUtil.retryUntilTrue( + () -> { + try { + StatusResponseHolder response = httpClient.go( + new Request(HttpMethod.GET, new URL(StringUtils.format("%s/status/health", host))), + StatusResponseHandler.getInstance() + ).get(); + + LOG.info("%s %s", response.getStatus(), response.getContent()); + return response.getStatus().equals(HttpResponseStatus.OK); + } + catch (Throwable e) { + // + // suppress stack trace logging for some specific exceptions + // to reduce excessive stack trace messages when waiting druid nodes to start up + // + if (e.getCause() instanceof ChannelException) { + Throwable channelException = e.getCause(); + + if (channelException.getCause() instanceof ClosedChannelException) { + LOG.error("Channel Closed"); + } else if ("Channel disconnected".equals(channelException.getMessage())) { + // log message only + LOG.error("Channel disconnected"); + } else { + // log stack trace for unknown exception + LOG.error(e, "Error while waiting for [%s] to be ready", host); + } + } else { + // log stack trace for unknown exception + LOG.error(e, "Error while waiting for [%s] to be ready", host); + } + + return false; + } + }, + "Waiting for instance to be ready: [" + host + "]" + ); + } + + private void postDynamicConfig(CoordinatorDynamicConfig coordinatorDynamicConfig) + { + ITRetryUtil.retryUntilTrue( + () -> { + try { + String url = StringUtils.format("%s/druid/coordinator/v1/config", config.getCoordinatorUrl()); + StatusResponseHolder response = httpClient.go( + new Request(HttpMethod.POST, new URL(url)).setContent( + "application/json", + jsonMapper.writeValueAsBytes(coordinatorDynamicConfig) + ), StatusResponseHandler.getInstance() + ).get(); + + LOG.info("%s %s", response.getStatus(), response.getContent()); + // if coordinator is not leader then it will return 307 instead of 200 + return response.getStatus().equals(HttpResponseStatus.OK) || response.getStatus().equals(HttpResponseStatus.TEMPORARY_REDIRECT); + } + catch (Throwable e) { + LOG.error(e, "Error while posting dynamic config"); + return false; + } + }, + "Posting dynamic config after startup" + ); + } +} diff --git a/integration-tests-ex/cases/src/test/resources/cluster/AzureDeepStorage/docker.yaml b/integration-tests-ex/cases/src/test/resources/cluster/AzureDeepStorage/docker.yaml new file mode 100644 index 00000000000..ac07bab9843 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/cluster/AzureDeepStorage/docker.yaml @@ -0,0 +1,58 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +#------------------------------------------------------------------------- + +# Definition of the batch index test cluster. +# See https://yaml.org/spec/1.2.2 for more about YAML +include: + - /cluster/Common/zk-metastore.yaml + +druid: + coordinator: + instances: + - port: 8081 + overlord: + instances: + - port: 8090 + broker: + instances: + - port: 8082 + router: + instances: + - port: 8888 + historical: + instances: + - port: 8083 + indexer: + instances: + - port: 8091 + +# Properties to be set in the Properties object used in +# Guice configuration in lieu of the server-side runtime.properties +# file. +# +# druid.global.http.numMaxThreads avoids creating 40+ Netty threads. +# We only ever use 1. +# druid.test.config.dockerIp is used by some older test code. Remove +# it when that code is updated. +properties: + druid.global.http.numMaxThreads: 3 + druid.broker.http.numMaxThreads: 3 + druid.test.config.dockerIp: localhost + druid.test.config.cloudBucket: "new-it-framework" + druid.test.config.cloudPath: "" + docker.build.hadoop: true + start.hadoop.docker: true + override.config.path: "/Users/abhishekagrawal/pr_druid_it/druid/integration-tests-ex/it-azure-deep-storage/azure-config" diff --git a/integration-tests-ex/cases/src/test/resources/cluster/BatchIndex/docker.yaml b/integration-tests-ex/cases/src/test/resources/cluster/BatchIndex/docker.yaml new file mode 100644 index 00000000000..d676f530e90 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/cluster/BatchIndex/docker.yaml @@ -0,0 +1,40 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +#------------------------------------------------------------------------- + +# Definition of the batch index test cluster. +# See https://yaml.org/spec/1.2.2 for more about YAML +include: + - /cluster/Common/zk-metastore.yaml + +druid: + coordinator: + instances: + - port: 8081 + overlord: + instances: + - port: 8090 + broker: + instances: + - port: 8082 + router: + instances: + - port: 8888 + historical: + instances: + - port: 8083 + indexer: + instances: + - port: 8091 diff --git a/integration-tests-ex/cases/src/test/resources/cluster/Common/kafka.yaml b/integration-tests-ex/cases/src/test/resources/cluster/Common/kafka.yaml new file mode 100644 index 00000000000..769c80989a4 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/cluster/Common/kafka.yaml @@ -0,0 +1,22 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# Definition of the Kafka node used by some integration tests. +# Include this into test-specific configurations. + +kafka: + instances: + - port: 9092 + proxyPort: 9093 diff --git a/integration-tests-ex/cases/src/test/resources/cluster/Common/zk-metastore.yaml b/integration-tests-ex/cases/src/test/resources/cluster/Common/zk-metastore.yaml new file mode 100644 index 00000000000..fded303c113 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/cluster/Common/zk-metastore.yaml @@ -0,0 +1,28 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# Defines the standard ZooKeeper and metadata storage (metastore) +# services used in all Druid test clusters. Include this in to +# test-specific configuration files. + +zk: + instances: + - port: 2181 +metastore: + connectURI: jdbc:mysql://:/druid + user: druid + password: diurd + instances: + - port: 3306 diff --git a/integration-tests-ex/cases/src/test/resources/cluster/HighAvailability/docker.yaml b/integration-tests-ex/cases/src/test/resources/cluster/HighAvailability/docker.yaml new file mode 100644 index 00000000000..5d99ffcbc52 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/cluster/HighAvailability/docker.yaml @@ -0,0 +1,204 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +#------------------------------------------------------------------------- + +# Definition of the high-availability test cluster. +# See https://yaml.org/spec/1.2.2 for more about YAML +include: + - /cluster/Common/zk-metastore.yaml + +# This test does not use ingest or query workers. +druid: + coordinator: + instances: + - tag: one + port: 8081 + - tag: two + port: 8081 + proxyPort: 18081 + overlord: + instances: + - tag: one + port: 8090 + - tag: two + port: 8090 + proxyPort: 18090 + broker: + instances: + - port: 8082 + router: + instances: + - port: 8888 + custom-node-role: + instances: + - port: 9301 + +# Metastore initialization queries. +# REPLACE is used so that the statements are idempotent +# The fancy formatting is for human consumption, it is compacted internally +metastoreInit: + - sql: | + REPLACE INTO druid_segments ( + id, dataSource, created_date, start, end, partitioned, version, used, payload + ) + VALUES ( + 'twitterstream_2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z_2013-01-02T04:13:41.980Z_v9', + 'twitterstream', + '2013-05-13T01:08:18.192Z', + '2013-01-01T00:00:00.000Z', + '2013-01-02T00:00:00.000Z', + 0, + '2013-01-02T04:13:41.980Z_v9', + 1, + '{"dataSource": "twitterstream", + "interval": "2013-01-01T00:00:00.000Z/2013-01-02T00:00:00.000Z", + "version": "2013-01-02T04:13:41.980Z_v9", + "loadSpec": { + "type": "s3_zip", + "bucket": "static.druid.io", + "key": "data/segments/twitterstream/2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z/2013-01-02T04:13:41.980Z_v9/0/index.zip" + }, + "dimensions": "has_links,first_hashtag,user_time_zone,user_location,has_mention,user_lang, + rt_name,user_name,is_retweet,is_viral,has_geo,url_domain,user_mention_name,reply_to_name", + "metrics": "count,tweet_length,num_followers,num_links,num_mentions,num_hashtags,num_favorites,user_total_tweets", + "shardSpec": {"type": "none"}, + "binaryVersion": 9, + "size": 445235220, + "identifier": "twitterstream_2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z_2013-01-02T04:13:41.980Z_v9" + }' + ) + + - sql: | + REPLACE INTO druid_segments ( + id, dataSource, created_date, start, end, partitioned, version, used, payload + ) + VALUES ( + 'twitterstream_2013-01-02T00:00:00.000Z_2013-01-03T00:00:00.000Z_2013-01-03T03:44:58.791Z_v9', + 'twitterstream', + '2013-05-13T00:03:28.640Z', + '2013-01-02T00:00:00.000Z', + '2013-01-03T00:00:00.000Z', + 0, + '2013-01-03T03:44:58.791Z_v9', + 1, + '{"dataSource": "twitterstream", + "interval": "2013-01-02T00:00:00.000Z/2013-01-03T00:00:00.000Z", + "version": "2013-01-03T03:44:58.791Z_v9", + "loadSpec": { + "type": "s3_zip", + "bucket": "static.druid.io", + "key": "data/segments/twitterstream/2013-01-02T00:00:00.000Z_2013-01-03T00:00:00.000Z/2013-01-03T03:44:58.791Z_v9/0/index.zip" + }, + "dimensions": "has_links,first_hashtag,user_time_zone,user_location,has_mention,user_lang, + rt_name,user_name,is_retweet,is_viral,has_geo,url_domain,user_mention_name,reply_to_name", + "metrics": "count,tweet_length,num_followers,num_links,num_mentions,num_hashtags,num_favorites,user_total_tweets", + "shardSpec": {"type": "none"}, + "binaryVersion": 9, + "size": 435325540, + "identifier": "twitterstream_2013-01-02T00:00:00.000Z_2013-01-03T00:00:00.000Z_2013-01-03T03:44:58.791Z_v9" + }' + ) + + - sql: | + REPLACE INTO druid_segments ( + id, dataSource, created_date, start, end, partitioned, version, used, payload + ) + VALUES ( + 'twitterstream_2013-01-03T00:00:00.000Z_2013-01-04T00:00:00.000Z_2013-01-04T04:09:13.590Z_v9', + 'twitterstream', + '2013-05-13T00:03:48.807Z', + '2013-01-03T00:00:00.000Z', + '2013-01-04T00:00:00.000Z', + 0, + '2013-01-04T04:09:13.590Z_v9', + 1, + '{"dataSource": "twitterstream", + "interval": "2013-01-03T00:00:00.000Z/2013-01-04T00:00:00.000Z", + "version": "2013-01-04T04:09:13.590Z_v9", + "loadSpec": { + "type": "s3_zip", + "bucket": "static.druid.io", + "key": "data/segments/twitterstream/2013-01-03T00:00:00.000Z_2013-01-04T00:00:00.000Z/2013-01-04T04:09:13.590Z_v9/0/index.zip" + }, + "dimensions": "has_links,first_hashtag,user_time_zone,user_location,has_mention,user_lang, + rt_name,user_name,is_retweet,is_viral,has_geo,url_domain,user_mention_name,reply_to_name", + "metrics": "count,tweet_length,num_followers,num_links,num_mentions,num_hashtags,num_favorites,user_total_tweets", + "shardSpec": {"type": "none"}, + "binaryVersion": 9, + "size": 411651320, + "identifier": "twitterstream_2013-01-03T00:00:00.000Z_2013-01-04T00:00:00.000Z_2013-01-04T04:09:13.590Z_v9" + }' + ) + + - sql: | + REPLACE INTO druid_segments ( + id, dataSource, created_date, start, end, partitioned, version, used, payload + ) + VALUES ( + 'wikipedia_editstream_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9', + 'wikipedia_editstream', + '2013-03-15T20:49:52.348Z', + '2012-12-29T00:00:00.000Z', + '2013-01-10T08:00:00.000Z', + 0, + '2013-01-10T08:13:47.830Z_v9', + 1, + '{"dataSource": "wikipedia_editstream", + "interval": "2012-12-29T00:00:00.000Z/2013-01-10T08:00:00.000Z", + "version": "2013-01-10T08:13:47.830Z_v9", + "loadSpec": { + "type": "s3_zip", + "bucket": "static.druid.io", + "key": "data/segments/wikipedia_editstream/2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z/2013-01-10T08:13:47.830Z_v9/0/index.zip"}, + "dimensions": "anonymous,area_code,city,continent_code,country_name,dma_code,geo,language, + namespace,network,newpage,page,postal_code,region_lookup,robot,unpatrolled,user", + "metrics": "added,count,deleted,delta,delta_hist,unique_users,variation", + "shardSpec": {"type": "none"}, + "binaryVersion": 9, + "size": 446027801, + "identifier": "wikipedia_editstream_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9" + }' + ) + + - sql: | + REPLACE INTO druid_segments ( + id, dataSource, created_date, start, end, partitioned, version, used, payload + ) + VALUES ( + 'wikipedia_2013-08-01T00:00:00.000Z_2013-08-02T00:00:00.000Z_2013-08-08T21:22:48.989Z', + 'wikipedia', + '2013-08-08T21:26:23.799Z', + '2013-08-01T00:00:00.000Z', + '2013-08-02T00:00:00.000Z', + 0, + '2013-08-08T21:22:48.989Z', + 1, + '{"dataSource": "wikipedia", + "interval": "2013-08-01T00:00:00.000Z/2013-08-02T00:00:00.000Z", + "version": "2013-08-08T21:22:48.989Z", + "loadSpec": { + "type": "s3_zip", + "bucket": "static.druid.io", + "key": "data/segments/wikipedia/20130801T000000.000Z_20130802T000000.000Z/2013-08-08T21_22_48.989Z/0/index.zip" + }, + "dimensions": "dma_code,continent_code,geo,area_code,robot,country_name,network,city,namespace, + anonymous,unpatrolled,page,postal_code,language,newpage,user,region_lookup", + "metrics": "count,delta,variation,added,deleted", + "shardSpec": {"type": "none"}, + "binaryVersion": 9, + "size": 24664730, + "identifier": "wikipedia_2013-08-01T00:00:00.000Z_2013-08-02T00:00:00.000Z_2013-08-08T21:22:48.989Z" + }' + ) diff --git a/integration-tests-ex/cases/src/test/resources/cluster/HighAvailability/queries/sys.json b/integration-tests-ex/cases/src/test/resources/cluster/HighAvailability/queries/sys.json new file mode 100644 index 00000000000..d5d60d4f297 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/cluster/HighAvailability/queries/sys.json @@ -0,0 +1,39 @@ +[ + { + "description": "query sys.servers to make sure all expected servers are available", + "query": { + "query": "SELECT host, server_type, is_leader FROM sys.servers ORDER BY host" + }, + "expectedResults": [ + {"host":"%%BROKER%%","server_type":"broker", "is_leader": %%NON_LEADER%%}, + {"host":"%%COORDINATOR_ONE%%","server_type":"coordinator", "is_leader": %%COORDINATOR_ONE_LEADER%%}, + {"host":"%%COORDINATOR_TWO%%","server_type":"coordinator", "is_leader": %%COORDINATOR_TWO_LEADER%%}, + {"host":"%%OVERLORD_ONE%%","server_type":"overlord", "is_leader": %%OVERLORD_ONE_LEADER%%}, + {"host":"%%OVERLORD_TWO%%","server_type":"overlord", "is_leader": %%OVERLORD_TWO_LEADER%%}, + {"host":"%%ROUTER%%","server_type":"router", "is_leader": %%NON_LEADER%%} + ] + }, + { + "description": "query sys.segments which is fed via coordinator data", + "query": { + "query": "SELECT datasource, count(*) FROM sys.segments WHERE datasource='wikipedia_editstream' OR datasource='twitterstream' GROUP BY 1 " + }, + "expectedResults": [ + { + "datasource": "wikipedia_editstream", + "EXPR$1": 1 + }, + { + "datasource": "twitterstream", + "EXPR$1": 3 + } + ] + }, + { + "description": "query sys.tasks which is fed via overlord", + "query": { + "query": "SELECT datasource, count(*) FROM sys.tasks WHERE datasource='wikipedia_editstream' OR datasource='twitterstream' GROUP BY 1 " + }, + "expectedResults": [] + } +] \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/config-test/test.yaml b/integration-tests-ex/cases/src/test/resources/config-test/test.yaml new file mode 100644 index 00000000000..92745caea11 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/config-test/test.yaml @@ -0,0 +1,85 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# Definition of a simple dummy "cluster" for use in +# testing the configuration mechanism itself. +# See https://yaml.org/spec/1.2.2 for more about YAML +include: + - /cluster/Common/zk-metastore.yaml + - /cluster/Common/kafka.yaml + +druid: + overlord: + instances: + - port: 8090 + coordinator: + instances: + - port: 8081 + historical: + instances: + - port: 8083 + middlemanager: + instances: + - port: 8091 + broker: + instances: + - port: 8082 + router: + instances: + - port: 8888 + +properties: + druid.test.config.dockerIp: localhost + druid.test.config.cloudRegion: myRegion + my.test.property: howdy + +settings: + cloudBucket: myBucket + cloud_path: myPath + cloud_region: hidden + druid_s3_accessKey: secret + +metastoreInit: + - sql: | + REPLACE INTO druid_segments ( + id, dataSource, created_date, start, end, partitioned, version, used, payload + ) + VALUES ( + 'twitterstream_2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z_2013-01-02T04:13:41.980Z_v9', + 'twitterstream', + '2013-05-13T01:08:18.192Z', + '2013-01-01T00:00:00.000Z', + '2013-01-02T00:00:00.000Z', + 0, + '2013-01-02T04:13:41.980Z_v9', + 1, + '{"dataSource": "twitterstream", + "interval": "2013-01-01T00:00:00.000Z/2013-01-02T00:00:00.000Z", + "version": "2013-01-02T04:13:41.980Z_v9", + "loadSpec": { + "type": "s3_zip", + "bucket": "static.druid.io", + "key": "data/segments/twitterstream/2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z/2013-01-02T04:13:41.980Z_v9/0/index.zip" + }, + "dimensions": "has_links,first_hashtag,user_time_zone,user_location,has_mention,user_lang,rt_name,user_name, + is_retweet,is_viral,has_geo,url_domain,user_mention_name,reply_to_name", + "metrics": "count,tweet_length,num_followers,num_links,num_mentions,num_hashtags,num_favorites, + user_total_tweets", + "shardSpec": {"type": "none"}, + "binaryVersion": 9, + "size": 445235220, + "identifier": "twitterstream_2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z_2013-01-02T04:13:41.980Z_v9" + }' + ) diff --git a/integration-tests-ex/cases/src/test/resources/indexer/broadcast_join_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/broadcast_join_index_task.json new file mode 100644 index 00000000000..20c3b162ea6 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/broadcast_join_index_task.json @@ -0,0 +1,82 @@ +{ + "type": "index_parallel", + "spec": { + "dataSchema": { + "dataSource": "%%JOIN_DATASOURCE%%", + "timestampSpec": { + "column": "timestamp", + "format": "iso" + }, + "dimensionsSpec": { + "dimensions": [ + "page", + "language", + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city", + { + "type": "long", + "name": "added" + }, + { + "type": "long", + "name": "deleted" + } + ] + }, + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "sum_added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "sum_deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + } + ], + "granularitySpec": { + "segmentGranularity": "YEAR", + "queryGranularity": "second" + } + }, + "ioConfig": { + "type": "index_parallel", + "inputSource": { + "type": "local", + "baseDir": "/resources/data/union_query/", + "filter": "wikipedia_index_data*" + }, + "appendToExisting": false, + "inputFormat": { + "type": "json" + } + }, + "tuningConfig": { + "type": "index_parallel", + "indexSpec": { + "segmentLoader": { + "type": "broadcastJoinableMMapSegmentFactory", + "keyColumns": ["user", "language", "added", "deleted"] + } + } + } + } +} \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/json_path_index_queries.json b/integration-tests-ex/cases/src/test/resources/indexer/json_path_index_queries.json new file mode 100644 index 00000000000..1940cc6ea02 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/json_path_index_queries.json @@ -0,0 +1,49 @@ +[ + { + "description": "timeseries", + "query": { + "queryType": "timeseries", + "dataSource": "json_path_index_test", + "intervals": [ + "1000/3000" + ], + "aggregations": [ + { + "type": "longSum", + "name": "len", + "fieldName": "len" + }, + { + "type": "longSum", + "name": "max", + "fieldName": "max" + }, + { + "type": "longSum", + "name": "min", + "fieldName": "min" + }, + { + "type": "longSum", + "name": "sum", + "fieldName": "sum" + } + ], + "granularity": { + "type": "all" + } + }, + "expectedResults": [ + { + "timestamp": "2013-08-31T01:02:33.000Z", + "result": { + "sum": 10, + "min": 0, + "len": 5, + "max": 4 + } + } + ] + } +] + diff --git a/integration-tests-ex/cases/src/test/resources/indexer/json_path_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/json_path_index_task.json new file mode 100644 index 00000000000..2fd6990b116 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/json_path_index_task.json @@ -0,0 +1,80 @@ +{ + "type": "index", + "dataSource": "json_path_index_test", + "spec": { + "dataSchema": { + "dataSource": "json_path_index_test", + "timestampSpec": { + "column": "timestamp", + "format": "iso" + }, + "dimensionsSpec": { + "dimensions": [ + { + "type": "long", + "name": "len" + }, + { + "type": "long", + "name": "min" + }, + { + "type": "long", + "name": "max" + }, + { + "type": "long", + "name": "sum" + } + ], + "dimensionExclusions": [ + "__time", + "timestamp" + ] + }, + "metricsSpec": [], + "granularitySpec": { + "type": "uniform", + "segmentGranularity": "HOUR", + "queryGranularity": { + "type": "none" + } + } + }, + "ioConfig": { + "type": "index", + "inputSource": { + "type": "inline", + "data": "{\"timestamp\": \"2013-08-31T01:02:33Z\", \"values\": [0,1,2,3,4] }" + }, + "inputFormat": { + "type": "json", + "flattenSpec": { + "useFieldDiscovery": true, + "fields": [ + { + "type": "path", + "name": "len", + "expr": "$.values.length()" + }, + { + "type": "path", + "name": "min", + "expr": "$.values.min()" + }, + { + "type": "path", + "name": "max", + "expr": "$.values.max()" + }, + { + "type": "path", + "name": "sum", + "expr": "$.values.sum()" + } + ] + } + } + } + } +} \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/segment_metadata_query.json b/integration-tests-ex/cases/src/test/resources/indexer/segment_metadata_query.json new file mode 100644 index 00000000000..1868c4596d0 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/segment_metadata_query.json @@ -0,0 +1,16 @@ +[ + { + "query": { + "queryType": "segmentMetadata", + "dataSource": "%%DATASOURCE%%", + "analysisTypes": [ + "%%ANALYSIS_TYPE%%" + ], + "intervals": [ + "%%INTERVALS%%" + ] + }, + "expectedResults": %%EXPECTED_QUERY_GRANULARITY%%, + "fieldsToTest": ["queryGranularity"] + } +] diff --git a/integration-tests-ex/cases/src/test/resources/indexer/sparse_column_index_queries.json b/integration-tests-ex/cases/src/test/resources/indexer/sparse_column_index_queries.json new file mode 100644 index 00000000000..193e69fbc9c --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/sparse_column_index_queries.json @@ -0,0 +1,80 @@ +[ + { + "description": "timeseries, 1 agg, all", + "query":{ + "queryType" : "timeBoundary", + "dataSource": "%%DATASOURCE%%" + }, + "expectedResults":[ + { + "timestamp" : "2015-09-12T00:00:00.000Z", + "result" : { + "minTime" : "2015-09-12T00:00:00.000Z", + "maxTime" : "2015-09-12T00:00:00.000Z" + } + } + ] + }, + { + "description": "scan, all", + "query": { + "queryType": "scan", + "dataSource": "%%DATASOURCE%%", + "intervals": [ + "2013-01-01/2020-01-02" + ], + "resultFormat":"compactedList" + }, + "expectedResults": %%EXPECTED_SCAN_RESULT%%, + "fieldsToTest": ["events"] + }, + { + "description": "roll up ratio", + "query": { + "queryType":"timeseries", + "dataSource":{ + "type":"table", + "name":"%%DATASOURCE%%" + }, + "intervals":{ + "type":"intervals", + "intervals":[ + "2013-01-01/2020-01-02" + ] + }, + "granularity":{ + "type":"all" + }, + "aggregations":[ + { + "type":"count", + "name":"a0" + }, + { + "type":"longSum", + "name":"a1", + "fieldName":"count", + "expression":null + } + ], + "postAggregations":[ + { + "type":"expression", + "name":"p0", + "expression":"((\"a0\" * 1.00) / \"a1\")", + "ordering":null + } + ] + }, + "expectedResults": [ + { + "timestamp" : "2015-09-12T00:00:00.000Z", + "result" : { + "a1" : %%EXPECTED_SUM_COUNT%%, + "p0" : %%EXPECTED_ROLLUP_RATIO%%, + "a0" : %%EXPECTED_NUM_ROW%% + } + } + ] + } +] \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/sparse_column_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/sparse_column_index_task.json new file mode 100644 index 00000000000..3a21a856ac6 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/sparse_column_index_task.json @@ -0,0 +1,57 @@ +{ + "type": "index_parallel", + "spec": { + "ioConfig": { + "type": "index_parallel", + "inputSource": { + "type": "inline", + "data": "{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"C\",\"dimB\":\"F\",\"metA\":1}\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"C\",\"dimB\":\"J\",\"metA\":1}\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"H\",\"dimB\":\"X\",\"metA\":1}\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"Z\",\"dimB\":\"S\",\"metA\":1}\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"H\",\"dimB\":\"X\",\"metA\":1}\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"H\",\"dimB\":\"Z\",\"metA\":1}\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"J\",\"dimB\":\"R\",\"metA\":1}\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"H\",\"dimB\":\"T\",\"metA\":1}\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimA\":\"H\",\"dimB\":\"X\",\"metA\":1}\n{\"time\":\"2015-09-12T00:46:58.771Z\",\"dimC\":\"A\",\"dimB\":\"X\",\"metA\":1}\n" + }, + "inputFormat": { + "type": "json" + } + }, + "tuningConfig": { + "type": "index_parallel", + "partitionsSpec": { + "type": "dynamic", + "maxRowsPerSegment": 3, + "maxTotalRows": 3 + }, + "maxRowsInMemory": 3 + }, + "dataSchema": { + "dataSource": "%%DATASOURCE%%", + "timestampSpec": { + "column": "time", + "format": "iso" + }, + "dimensionsSpec": { + "dimensions": [ + "dimB", + "dimA", + "dimC", + "dimD", + "dimE", + "dimF" + ] + }, + "granularitySpec": { + "queryGranularity": "hour", + "rollup": true, + "segmentGranularity": "hour" + }, + "metricsSpec": [ + { + "name": "count", + "type": "count" + }, + { + "name": "sum_metA", + "type": "longSum", + "fieldName": "metA" + } + ] + } + } +} \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/sparse_column_with_dim_compaction_task.json b/integration-tests-ex/cases/src/test/resources/indexer/sparse_column_with_dim_compaction_task.json new file mode 100644 index 00000000000..9416a3f6bda --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/sparse_column_with_dim_compaction_task.json @@ -0,0 +1,19 @@ +{ + "type": "compact", + "dataSource": "%%DATASOURCE%%", + "dimensionsSpec": { + "dimensions": %%DIMENSION_NAMES%% + }, + "interval": "2010-10-29T05:00:00Z/2030-10-29T06:00:00Z", + "tuningConfig": { + "type": "index_parallel", + "maxRowsPerSegment": 3, + "maxRowsInMemory": 3, + "maxNumConcurrentSubTasks": 2, + "partitionsSpec": { + "type": "hashed", + "numShards": 1 + }, + "forceGuaranteedRollup": true + } +} \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/sparse_column_without_dim_compaction_task.json b/integration-tests-ex/cases/src/test/resources/indexer/sparse_column_without_dim_compaction_task.json new file mode 100644 index 00000000000..a149d7a2512 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/sparse_column_without_dim_compaction_task.json @@ -0,0 +1,16 @@ +{ + "type": "compact", + "dataSource": "%%DATASOURCE%%", + "interval": "2010-10-29T05:00:00Z/2030-10-29T06:00:00Z", + "tuningConfig": { + "type": "index_parallel", + "maxRowsPerSegment": 3, + "maxRowsInMemory": 3, + "maxNumConcurrentSubTasks": 2, + "partitionsSpec": { + "type": "hashed", + "numShards": 1 + }, + "forceGuaranteedRollup": true + } +} \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/sys_segment_batch_index_queries.json b/integration-tests-ex/cases/src/test/resources/indexer/sys_segment_batch_index_queries.json new file mode 100644 index 00000000000..931ad895778 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/sys_segment_batch_index_queries.json @@ -0,0 +1,35 @@ +[ + { + "query": { + "query": "SELECT count(*) FROM sys.segments WHERE datasource LIKE 'wikipedia_index_test%'" + }, + "expectedResults": [ + { + "EXPR$0": 4 + } + ] + }, + { + "query": { + "query": "SELECT server_type FROM sys.servers WHERE tier IS NOT NULL AND server_type <> 'indexer'" + }, + "expectedResults": [ + { + "server_type":"historical" + }, + { + "server_type":"broker" + } + ] + }, + { + "query": { + "query": "SELECT status AS status FROM sys.tasks WHERE datasource LIKE 'wikipedia_index_test%' GROUP BY 1" + }, + "expectedResults": [ + { + "status": "SUCCESS" + } + ] + } +] \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_cloud_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_cloud_index_task.json new file mode 100644 index 00000000000..8c4242c91e2 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_cloud_index_task.json @@ -0,0 +1,87 @@ +{ + "type": "index_parallel", + "spec": { + "dataSchema": { + "dataSource": "%%DATASOURCE%%", + "timestampSpec": { + "column": "timestamp" + }, + "dimensionsSpec": { + "dimensions": [ + "page", + {"type": "string", "name": "language", "createBitmapIndex": false}, + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city" + ] + }, + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + }, + { + "name": "thetaSketch", + "type": "thetaSketch", + "fieldName": "user" + }, + { + "name": "quantilesDoublesSketch", + "type": "quantilesDoublesSketch", + "fieldName": "delta" + }, + { + "name": "HLLSketchBuild", + "type": "HLLSketchBuild", + "fieldName": "user" + } + ], + "granularitySpec": { + "segmentGranularity": "DAY", + "queryGranularity": "second", + "intervals" : [ "2013-08-31/2013-09-02" ] + } + }, + "ioConfig": { + "type": "index_parallel", + "inputSource": { + "type": "%%INPUT_SOURCE_TYPE%%", + "%%INPUT_SOURCE_PROPERTY_KEY%%": %%INPUT_SOURCE_PROPERTY_VALUE%% + }, + "inputFormat": { + "type": "%%INPUT_FORMAT_TYPE%%" + } + }, + "tuningConfig": { + "type": "index_parallel", + "maxNumConcurrentSubTasks": 10, + "partitionsSpec": %%PARTITIONS_SPEC%%, + "splitHintSpec": { + "type": "maxSize", + "maxNumFiles": 1 + } + } + } +} \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_cloud_simple_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_cloud_simple_index_task.json new file mode 100644 index 00000000000..742ab209f72 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_cloud_simple_index_task.json @@ -0,0 +1,81 @@ +{ + "type": "index", + "spec": { + "dataSchema": { + "dataSource": "%%DATASOURCE%%", + "timestampSpec": { + "column": "timestamp" + }, + "dimensionsSpec": { + "dimensions": [ + "page", + {"type": "string", "name": "language", "createBitmapIndex": false}, + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city" + ] + }, + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + }, + { + "name": "thetaSketch", + "type": "thetaSketch", + "fieldName": "user" + }, + { + "name": "quantilesDoublesSketch", + "type": "quantilesDoublesSketch", + "fieldName": "delta" + }, + { + "name": "HLLSketchBuild", + "type": "HLLSketchBuild", + "fieldName": "user" + } + ], + "granularitySpec": { + "segmentGranularity": "DAY", + "queryGranularity": "second", + "intervals" : [ "2013-08-31/2013-09-02" ] + } + }, + "ioConfig": { + "type": "index", + "inputSource": { + "type": "%%INPUT_SOURCE_TYPE%%", + "%%INPUT_SOURCE_PROPERTY_KEY%%": %%INPUT_SOURCE_PROPERTY_VALUE%% + }, + "inputFormat": { + "type": "json" + } + }, + "tuningConfig": { + "type": "index" + } + } +} \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_combining_firehose_index_queries.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_combining_firehose_index_queries.json new file mode 100644 index 00000000000..302d2fea284 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_combining_firehose_index_queries.json @@ -0,0 +1,141 @@ +[ + { + "description": "timeseries, 1 agg, all", + "query": { + "queryType": "timeBoundary", + "dataSource": "%%DATASOURCE%%" + }, + "expectedResults": [ + { + "timestamp": "2013-08-31T01:02:33.000Z", + "result": { + "minTime": "2013-08-31T01:02:33.000Z", + "maxTime": "2013-09-01T18:22:39.000Z" + } + } + ] + }, + { + "description": "timeseries, datasketch aggs, all", + "query": { + "queryType": "timeseries", + "dataSource": "%%DATASOURCE%%", + "granularity": "day", + "intervals": [ + "2013-09-01T00:00/2013-09-02T00:00" + ], + "filter": null, + "aggregations": [ + { + "type": "HLLSketchMerge", + "name": "approxCountHLL", + "fieldName": "HLLSketchBuild", + "lgK": 12, + "tgtHllType": "HLL_4", + "round": true + }, + { + "type": "thetaSketch", + "name": "approxCountTheta", + "fieldName": "thetaSketch", + "size": 16384, + "shouldFinalize": true, + "isInputThetaSketch": false, + "errorBoundsStdDev": null + }, + { + "type": "quantilesDoublesSketch", + "name": "quantilesSketch", + "fieldName": "quantilesDoublesSketch", + "k": 128 + } + ] + }, + "expectedResults": [ + { + "timestamp": "2013-09-01T00:00:00.000Z", + "result": { + "quantilesSketch": 6, + "approxCountTheta": 6.0, + "approxCountHLL": 6 + } + } + ] + }, + { + "description": "having spec on post aggregation", + "query": { + "queryType": "groupBy", + "dataSource": "%%DATASOURCE%%", + "granularity": "day", + "dimensions": [ + "page" + ], + "filter": { + "type": "selector", + "dimension": "language", + "value": "zh" + }, + "aggregations": [ + { + "type": "count", + "name": "rows" + }, + { + "type": "longSum", + "fieldName": "added", + "name": "added_count" + } + ], + "postAggregations": [ + { + "type": "arithmetic", + "name": "added_count_times_ten", + "fn": "*", + "fields": [ + { + "type": "fieldAccess", + "name": "added_count", + "fieldName": "added_count" + }, + { + "type": "constant", + "name": "const", + "value": 10 + } + ] + } + ], + "having": { + "type": "greaterThan", + "aggregation": "added_count_times_ten", + "value": 9000 + }, + "intervals": [ + "2013-08-31T00:00/2013-09-01T00:00" + ] + }, + "expectedResults": [ + { + "version": "v1", + "timestamp": "2013-08-31T00:00:00.000Z", + "event": { + "added_count_times_ten": 9050.0, + "page": "Crimson Typhoon", + "added_count": 905, + "rows": 1 + } + }, + { + "version": "v1", + "timestamp": "2013-08-31T00:00:00.000Z", + "event": { + "added_count_times_ten": 9770.0, + "page": "Gypsy Danger", + "added_count": 977, + "rows": 1 + } + } + ] + } +] \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_combining_firehose_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_combining_firehose_index_task.json new file mode 100644 index 00000000000..1e7deffc6b5 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_combining_firehose_index_task.json @@ -0,0 +1,95 @@ +{ + "type": "index", + "spec": { + "dataSchema": { + "dataSource": "%%DATASOURCE%%", + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + }, + { + "name": "thetaSketch", + "type": "thetaSketch", + "fieldName": "user" + }, + { + "name": "quantilesDoublesSketch", + "type": "quantilesDoublesSketch", + "fieldName": "delta" + }, + { + "name": "HLLSketchBuild", + "type": "HLLSketchBuild", + "fieldName": "user" + } + ], + "granularitySpec": { + "segmentGranularity": "DAY", + "queryGranularity": "second", + "intervals" : [ "2013-08-31/2013-09-02" ] + }, + "parser": { + "parseSpec": { + "format" : "json", + "timestampSpec": { + "column": "timestamp" + }, + "dimensionsSpec": { + "dimensions": [ + "page", + {"type": "string", "name": "language", "createBitmapIndex": false}, + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city" + ] + } + } + } + }, + "ioConfig": { + "type": "index", + "firehose": { + "type": "combining", + "delegates": [ + { + "type": "local", + "baseDir": "/resources/indexer", + "filter": "wikipedia_combining_index_data.json" + }, + { + "type": "ingestSegment", + "dataSource": "%%COMBINING_DATASOURCE%%", + "interval": "2013-08-31/2013-09-02" + } + ] + } + }, + "tuningConfig": { + "type": "index", + "maxRowsPerSegment": 3 + } + } +} diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_combining_input_source_index_parallel_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_combining_input_source_index_parallel_task.json new file mode 100644 index 00000000000..8e1d0941bf0 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_combining_input_source_index_parallel_task.json @@ -0,0 +1,98 @@ +{ + "type": "index_parallel", + "spec": { + "dataSchema": { + "dataSource": "%%DATASOURCE%%", + "timestampSpec": { + "column": "timestamp" + }, + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + }, + { + "name": "thetaSketch", + "type": "thetaSketch", + "fieldName": "user" + }, + { + "name": "quantilesDoublesSketch", + "type": "quantilesDoublesSketch", + "fieldName": "delta" + }, + { + "name": "HLLSketchBuild", + "type": "HLLSketchBuild", + "fieldName": "user" + } + ], + "granularitySpec": { + "segmentGranularity": "DAY", + "queryGranularity": "second", + "intervals" : [ "2013-08-31/2013-09-02" ] + }, + "dimensionsSpec": { + "dimensions": [ + "page", + {"type": "string", "name": "language", "createBitmapIndex": false}, + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city" + ] + } + }, + "ioConfig": { + "type": "index_parallel", + "inputSource": { + "type": "combining", + "delegates": [ + { + "type": "local", + "baseDir": "/resources/indexer", + "filter": "wikipedia_combining_index_data.json" + }, + { + "type": "druid", + "dataSource": "%%COMBINING_DATASOURCE%%", + "interval": "2013-08-31/2013-09-02" + } + ] + }, + "appendToExisting": %%APPEND_TO_EXISTING%%, + "inputFormat": %%INPUT_FORMAT%% + }, + "tuningConfig": { + "type": "index_parallel", + "maxNumConcurrentSubTasks": 4, + "splitHintSpec": { + "type": "maxSize", + "maxNumFiles": 1 + }, + "forceGuaranteedRollup": %%FORCE_GUARANTEED_ROLLUP%%, + "partitionsSpec": %%PARTITIONS_SPEC%% + } + } +} diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_compaction_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_compaction_task.json new file mode 100644 index 00000000000..fb620c11aa2 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_compaction_task.json @@ -0,0 +1,14 @@ +{ + "type" : "compact", + "dataSource" : "%%DATASOURCE%%", + "ioConfig" : { + "type": "compact", + "inputSpec": { + "type": "interval", + "interval": "2013-08-31/2013-09-02" + } + }, + "context" : { + "storeCompactionState" : true + } +} \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_compaction_task_with_granularity_spec.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_compaction_task_with_granularity_spec.json new file mode 100644 index 00000000000..4fddeaa8aa3 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_compaction_task_with_granularity_spec.json @@ -0,0 +1,15 @@ +{ + "type" : "compact", + "dataSource" : "%%DATASOURCE%%", + "ioConfig" : { + "type": "compact", + "inputSpec": { + "type": "interval", + "interval": "2013-08-31/2013-09-02" + } + }, + "granularitySpec": %%GRANULARITY_SPEC%%, + "context" : { + "storeCompactionState" : true + } +} \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_compaction_task_with_segment_granularity.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_compaction_task_with_segment_granularity.json new file mode 100644 index 00000000000..7a2c016dc9d --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_compaction_task_with_segment_granularity.json @@ -0,0 +1,15 @@ +{ + "type" : "compact", + "dataSource" : "%%DATASOURCE%%", + "ioConfig" : { + "type": "compact", + "inputSpec": { + "type": "interval", + "interval": "2013-08-31/2013-09-02" + } + }, + "segmentGranularity": "%%SEGMENT_GRANULARITY%%", + "context" : { + "storeCompactionState" : true + } +} \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_double_ingestion_non_perfect_rollup_index_queries.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_double_ingestion_non_perfect_rollup_index_queries.json new file mode 100644 index 00000000000..586da63e3db --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_double_ingestion_non_perfect_rollup_index_queries.json @@ -0,0 +1,143 @@ +[ + { + "description": "timeseries, 1 agg, all", + "query":{ + "queryType" : "timeBoundary", + "dataSource": "%%DATASOURCE%%" + }, + "expectedResults":[ + { + "timestamp" : "2013-08-31T01:02:33.000Z", + "result" : { + "minTime" : "2013-08-31T01:02:33.000Z", + "maxTime" : "2013-09-01T12:41:27.000Z" + } + } + ] + }, + { + "description": "timeseries, datasketch aggs, all", + "query":{ + "queryType" : "timeseries", + "dataSource": "%%DATASOURCE%%", + "granularity":"day", + "intervals":[ + "2013-08-31T00:00/2013-09-01T00:00" + ], + "filter":null, + "aggregations":[ + { + "type": "HLLSketchMerge", + "name": "approxCountHLL", + "fieldName": "HLLSketchBuild", + "lgK": 12, + "tgtHllType": "HLL_4", + "round": true + }, + { + "type":"thetaSketch", + "name":"approxCountTheta", + "fieldName":"thetaSketch", + "size":16384, + "shouldFinalize":true, + "isInputThetaSketch":false, + "errorBoundsStdDev":null + }, + { + "type":"quantilesDoublesSketch", + "name":"quantilesSketch", + "fieldName":"quantilesDoublesSketch", + "k":128 + } + ] + }, + "expectedResults":[ + { + "timestamp" : "2013-08-31T00:00:00.000Z", + "result" : { + "quantilesSketch":10, + "approxCountTheta":5.0, + "approxCountHLL":5 + } + } + ] + }, + { + "description":"having spec on post aggregation", + "query":{ + "queryType":"groupBy", + "dataSource":"%%DATASOURCE%%", + "granularity":"day", + "dimensions":[ + "page" + ], + "filter":{ + "type":"selector", + "dimension":"language", + "value":"zh" + }, + "aggregations":[ + { + "type":"count", + "name":"rows" + }, + { + "type":"longSum", + "fieldName":"added", + "name":"added_count" + } + ], + "postAggregations": [ + { + "type":"arithmetic", + "name":"added_count_times_ten", + "fn":"*", + "fields":[ + {"type":"fieldAccess", "name":"added_count", "fieldName":"added_count"}, + {"type":"constant", "name":"const", "value":10} + ] + } + ], + "having":{"type":"greaterThan", "aggregation":"added_count_times_ten", "value":9000}, + "intervals":[ + "2013-08-31T00:00/2013-09-01T00:00" + ] + }, + "expectedResults":[ { + "version" : "v1", + "timestamp" : "2013-08-31T00:00:00.000Z", + "event" : { + "added_count_times_ten" : 18100.0, + "page" : "Crimson Typhoon", + "added_count" : 1810, + "rows" : 2 + } + } ] + }, + { + "description": "timeseries, count aggs, all", + "query":{ + "queryType" : "timeseries", + "dataSource": "%%DATASOURCE%%", + "granularity":"day", + "intervals":[ + "2013-08-31T00:00/2013-09-01T00:00" + ], + "filter":null, + "aggregations":[ + { + "type": "count", + "name": "rows" + } + ] + }, + "expectedResults":[ + { + "timestamp" : "2013-08-31T00:00:00.000Z", + "result" : { + "rows":10 + } + } + ] + } +] \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_double_ingestion_perfect_rollup_index_queries.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_double_ingestion_perfect_rollup_index_queries.json new file mode 100644 index 00000000000..eaa9592ca26 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_double_ingestion_perfect_rollup_index_queries.json @@ -0,0 +1,143 @@ +[ + { + "description": "timeseries, 1 agg, all", + "query":{ + "queryType" : "timeBoundary", + "dataSource": "%%DATASOURCE%%" + }, + "expectedResults":[ + { + "timestamp" : "2013-08-31T01:02:33.000Z", + "result" : { + "minTime" : "2013-08-31T01:02:33.000Z", + "maxTime" : "2013-09-01T12:41:27.000Z" + } + } + ] + }, + { + "description": "timeseries, datasketch aggs, all", + "query":{ + "queryType" : "timeseries", + "dataSource": "%%DATASOURCE%%", + "granularity":"day", + "intervals":[ + "2013-08-31T00:00/2013-09-01T00:00" + ], + "filter":null, + "aggregations":[ + { + "type": "HLLSketchMerge", + "name": "approxCountHLL", + "fieldName": "HLLSketchBuild", + "lgK": 12, + "tgtHllType": "HLL_4", + "round": true + }, + { + "type":"thetaSketch", + "name":"approxCountTheta", + "fieldName":"thetaSketch", + "size":16384, + "shouldFinalize":true, + "isInputThetaSketch":false, + "errorBoundsStdDev":null + }, + { + "type":"quantilesDoublesSketch", + "name":"quantilesSketch", + "fieldName":"quantilesDoublesSketch", + "k":128 + } + ] + }, + "expectedResults":[ + { + "timestamp" : "2013-08-31T00:00:00.000Z", + "result" : { + "quantilesSketch":10, + "approxCountTheta":5.0, + "approxCountHLL":5 + } + } + ] + }, + { + "description":"having spec on post aggregation", + "query":{ + "queryType":"groupBy", + "dataSource":"%%DATASOURCE%%", + "granularity":"day", + "dimensions":[ + "page" + ], + "filter":{ + "type":"selector", + "dimension":"language", + "value":"zh" + }, + "aggregations":[ + { + "type":"count", + "name":"rows" + }, + { + "type":"longSum", + "fieldName":"added", + "name":"added_count" + } + ], + "postAggregations": [ + { + "type":"arithmetic", + "name":"added_count_times_ten", + "fn":"*", + "fields":[ + {"type":"fieldAccess", "name":"added_count", "fieldName":"added_count"}, + {"type":"constant", "name":"const", "value":10} + ] + } + ], + "having":{"type":"greaterThan", "aggregation":"added_count_times_ten", "value":9000}, + "intervals":[ + "2013-08-31T00:00/2013-09-01T00:00" + ] + }, + "expectedResults":[ { + "version" : "v1", + "timestamp" : "2013-08-31T00:00:00.000Z", + "event" : { + "added_count_times_ten" : 18100.0, + "page" : "Crimson Typhoon", + "added_count" : 1810, + "rows" : 1 + } + } ] + }, + { + "description": "timeseries, count aggs, all", + "query":{ + "queryType" : "timeseries", + "dataSource": "%%DATASOURCE%%", + "granularity":"day", + "intervals":[ + "2013-08-31T00:00/2013-09-01T00:00" + ], + "filter":null, + "aggregations":[ + { + "type": "count", + "name": "rows" + } + ] + }, + "expectedResults":[ + { + "timestamp" : "2013-08-31T00:00:00.000Z", + "result" : { + "rows":5 + } + } + ] + } +] \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_http_inputsource_queries.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_http_inputsource_queries.json new file mode 100644 index 00000000000..f0cbb1cb859 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_http_inputsource_queries.json @@ -0,0 +1,47 @@ +[ + { + "description": "timeseries, 1 agg, all", + "query":{ + "queryType" : "timeBoundary", + "dataSource": "%%DATASOURCE%%" + }, + "expectedResults":[ + { + "timestamp" : "2016-06-27T00:00:11.000Z", + "result" : { + "minTime" : "2016-06-27T00:00:11.000Z", + "maxTime" : "2016-06-27T21:31:02.000Z" + } + } + ] + }, + { + "description": "simple aggr", + "query":{ + "queryType" : "topN", + "dataSource" : "%%DATASOURCE%%", + "intervals" : ["2016-06-27/2016-06-28"], + "granularity" : "all", + "dimension" : "page", + "metric" : "count", + "threshold" : 3, + "aggregations" : [ + { + "type" : "count", + "name" : "count" + } + ] + }, + "expectedResults":[ + { + "timestamp" : "2016-06-27T00:00:11.000Z", + "result" : + [ + {"count":58,"page":"Copa América Centenario"}, + {"count":32,"page":"User:Cyde/List of candidates for speedy deletion/Subpage"}, + {"count":32,"page":"Wikipedia:Administrators' noticeboard/Incidents"} + ] + } + ] + } +] \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_http_inputsource_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_http_inputsource_task.json new file mode 100644 index 00000000000..0ac4e05af85 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_http_inputsource_task.json @@ -0,0 +1,74 @@ +{ + "type": "index_parallel", + "spec": { + "dataSchema": { + "dataSource": "%%DATASOURCE%%", + "timestampSpec": { + "column": "timestamp" + }, + "dimensionsSpec": { + "dimensions": [ + "page", + {"type": "string", "name": "language", "createBitmapIndex": false}, + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city" + ] + }, + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + } + ], + "granularitySpec": { + "segmentGranularity": "DAY", + "queryGranularity": "second", + "intervals" : [ "2016-06/P1M" ] + } + }, + "ioConfig": { + "type": "index_parallel", + "inputSource": { + "type": "http", + "uris": ["https://druid.apache.org/data/wikipedia.json.gz", "https://druid.apache.org/data/wikipedia.json.gz"] + }, + "inputFormat": { + "type": "json" + } + }, + "tuningConfig": { + "type": "index_parallel", + "maxNumConcurrentSubTasks": 10, + "partitionsSpec": { + "type": "dynamic" + }, + "splitHintSpec": { + "type": "maxSize", + "maxNumFiles": 1 + } + } + } +} \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_queries.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_queries.json new file mode 100644 index 00000000000..928effe65e9 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_queries.json @@ -0,0 +1,150 @@ +[ + { + "description": "timeseries, 1 agg, all", + "query":{ + "queryType" : "timeBoundary", + "dataSource": "%%DATASOURCE%%" + }, + "expectedResults":[ + { + "timestamp" : "2013-08-31T01:02:33.000Z", + "result" : { + "minTime" : "2013-08-31T01:02:33.000Z", + "maxTime" : "2013-09-01T12:41:27.000Z" + } + } + ] + }, + { + "description": "timeseries, datasketch aggs, all", + "query":{ + "queryType" : "timeseries", + "dataSource": "%%DATASOURCE%%", + "granularity":"day", + "intervals":[ + "2013-08-31T00:00/2013-09-01T00:00" + ], + "filter":null, + "aggregations":[ + { + "type": "HLLSketchMerge", + "name": "approxCountHLL", + "fieldName": "HLLSketchBuild", + "lgK": 12, + "tgtHllType": "HLL_4", + "round": true + }, + { + "type":"thetaSketch", + "name":"approxCountTheta", + "fieldName":"thetaSketch", + "size":16384, + "shouldFinalize":true, + "isInputThetaSketch":false, + "errorBoundsStdDev":null + }, + { + "type":"quantilesDoublesSketch", + "name":"quantilesSketch", + "fieldName":"quantilesDoublesSketch", + "k":128 + } + ] + }, + "expectedResults":[ + { + "timestamp" : "2013-08-31T00:00:00.000Z", + "result" : { + "quantilesSketch":5, + "approxCountTheta":5.0, + "approxCountHLL":5 + } + } + ] + }, + { + "description":"having spec on post aggregation", + "query":{ + "queryType":"groupBy", + "dataSource":"%%DATASOURCE%%", + "granularity":"day", + "dimensions":[ + "page" + ], + "filter":{ + "type":"selector", + "dimension":"language", + "value":"zh" + }, + "aggregations":[ + { + "type":"count", + "name":"rows" + }, + { + "type":"longSum", + "fieldName":"added", + "name":"added_count" + } + ], + "postAggregations": [ + { + "type":"arithmetic", + "name":"added_count_times_ten", + "fn":"*", + "fields":[ + {"type":"fieldAccess", "name":"added_count", "fieldName":"added_count"}, + {"type":"constant", "name":"const", "value":10} + ] + } + ], + "having":{"type":"greaterThan", "aggregation":"added_count_times_ten", "value":9000}, + "intervals":[ + "2013-08-31T00:00/2013-09-01T00:00" + ] + }, + "expectedResults":[ { + "version" : "v1", + "timestamp" : "2013-08-31T00:00:00.000Z", + "event" : { + "added_count_times_ten" : 9050.0, + "page" : "Crimson Typhoon", + "added_count" : 905, + "rows" : 1 + } + } ] + }, + { + "description": "timeseries, stringFirst/stringLast aggs, all", + "query":{ + "queryType" : "timeseries", + "dataSource": "%%DATASOURCE%%", + "granularity":"day", + "intervals":[ + "2013-08-31T00:00/2013-09-01T00:00" + ], + "filter":null, + "aggregations":[ + { + "type": "stringFirst", + "name": "first_user", + "fieldName": "user" + }, + { + "type":"stringLast", + "name":"last_user", + "fieldName":"user" + } + ] + }, + "expectedResults":[ + { + "timestamp" : "2013-08-31T00:00:00.000Z", + "result" : { + "first_user":"nuclear", + "last_user":"stringer" + } + } + ] + } +] \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_queries_hour_query_granularity.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_queries_hour_query_granularity.json new file mode 100644 index 00000000000..6ed715b89eb --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_queries_hour_query_granularity.json @@ -0,0 +1,150 @@ +[ + { + "description": "timeseries, 1 agg, all", + "query":{ + "queryType" : "timeBoundary", + "dataSource": "%%DATASOURCE%%" + }, + "expectedResults":[ + { + "timestamp" : "2013-08-31T01:00:00.000Z", + "result" : { + "minTime" : "2013-08-31T01:00:00.000Z", + "maxTime" : "2013-09-01T12:00:00.000Z" + } + } + ] + }, + { + "description": "timeseries, datasketch aggs, all", + "query":{ + "queryType" : "timeseries", + "dataSource": "%%DATASOURCE%%", + "granularity":"day", + "intervals":[ + "2013-08-31T00:00/2013-09-01T00:00" + ], + "filter":null, + "aggregations":[ + { + "type": "HLLSketchMerge", + "name": "approxCountHLL", + "fieldName": "HLLSketchBuild", + "lgK": 12, + "tgtHllType": "HLL_4", + "round": true + }, + { + "type":"thetaSketch", + "name":"approxCountTheta", + "fieldName":"thetaSketch", + "size":16384, + "shouldFinalize":true, + "isInputThetaSketch":false, + "errorBoundsStdDev":null + }, + { + "type":"quantilesDoublesSketch", + "name":"quantilesSketch", + "fieldName":"quantilesDoublesSketch", + "k":128 + } + ] + }, + "expectedResults":[ + { + "timestamp" : "2013-08-31T00:00:00.000Z", + "result" : { + "quantilesSketch":5, + "approxCountTheta":5.0, + "approxCountHLL":5 + } + } + ] + }, + { + "description":"having spec on post aggregation", + "query":{ + "queryType":"groupBy", + "dataSource":"%%DATASOURCE%%", + "granularity":"day", + "dimensions":[ + "page" + ], + "filter":{ + "type":"selector", + "dimension":"language", + "value":"zh" + }, + "aggregations":[ + { + "type":"count", + "name":"rows" + }, + { + "type":"longSum", + "fieldName":"added", + "name":"added_count" + } + ], + "postAggregations": [ + { + "type":"arithmetic", + "name":"added_count_times_ten", + "fn":"*", + "fields":[ + {"type":"fieldAccess", "name":"added_count", "fieldName":"added_count"}, + {"type":"constant", "name":"const", "value":10} + ] + } + ], + "having":{"type":"greaterThan", "aggregation":"added_count_times_ten", "value":9000}, + "intervals":[ + "2013-08-31T00:00/2013-09-01T00:00" + ] + }, + "expectedResults":[ { + "version" : "v1", + "timestamp" : "2013-08-31T00:00:00.000Z", + "event" : { + "added_count_times_ten" : 9050.0, + "page" : "Crimson Typhoon", + "added_count" : 905, + "rows" : 1 + } + } ] + }, + { + "description": "timeseries, stringFirst/stringLast aggs, all", + "query":{ + "queryType" : "timeseries", + "dataSource": "%%DATASOURCE%%", + "granularity":"day", + "intervals":[ + "2013-08-31T00:00/2013-09-01T00:00" + ], + "filter":null, + "aggregations":[ + { + "type": "stringFirst", + "name": "first_user", + "fieldName": "user" + }, + { + "type":"stringLast", + "name":"last_user", + "fieldName":"user" + } + ] + }, + "expectedResults":[ + { + "timestamp" : "2013-08-31T00:00:00.000Z", + "result" : { + "first_user":"nuclear", + "last_user":"stringer" + } + } + ] + } +] \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_queries_with_transform.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_queries_with_transform.json new file mode 100644 index 00000000000..f0cfba67735 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_queries_with_transform.json @@ -0,0 +1,62 @@ +[ + { + "description":"having spec on post aggregation", + "query":{ + "queryType":"groupBy", + "dataSource":"%%DATASOURCE%%", + "granularity":"day", + "dimensions":[ + "page", + "city" + ], + "filter":{ + "type":"selector", + "dimension":"language", + "value":"language-zh" + }, + "aggregations":[ + { + "type":"count", + "name":"rows" + }, + { + "type":"longSum", + "fieldName":"triple-added", + "name":"added_count" + }, + { + "type":"longSum", + "fieldName":"delta", + "name":"delta_sum" + } + ], + "postAggregations": [ + { + "type":"arithmetic", + "name":"added_count_times_ten", + "fn":"*", + "fields":[ + {"type":"fieldAccess", "name":"added_count", "fieldName":"added_count"}, + {"type":"constant", "name":"const", "value":10} + ] + } + ], + "having":{"type":"greaterThan", "aggregation":"added_count_times_ten", "value":9000}, + "intervals":[ + "2013-08-31T00:00/2013-09-01T00:00" + ] + }, + "expectedResults":[ { + "version" : "v1", + "timestamp" : "2013-08-31T00:00:00.000Z", + "event" : { + "added_count_times_ten" : 27150.0, + "page" : "Crimson Typhoon", + "city" : "Taiyuan", + "added_count" : 2715, + "delta_sum" : 900, + "rows" : 1 + } + } ] + } +] \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_queries_year_query_granularity.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_queries_year_query_granularity.json new file mode 100644 index 00000000000..25f9203ac1b --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_queries_year_query_granularity.json @@ -0,0 +1,150 @@ +[ + { + "description": "timeseries, 1 agg, all", + "query":{ + "queryType" : "timeBoundary", + "dataSource": "%%DATASOURCE%%" + }, + "expectedResults":[ + { + "timestamp" : "2013-01-01T00:00:00.000Z", + "result" : { + "minTime" : "2013-01-01T00:00:00.000Z", + "maxTime" : "2013-01-01T00:00:00.000Z" + } + } + ] + }, + { + "description": "timeseries, datasketch aggs, all", + "query":{ + "queryType" : "timeseries", + "dataSource": "%%DATASOURCE%%", + "granularity":"day", + "intervals":[ + "2013-01-01T00:00/2014-01-01T00:00" + ], + "filter":null, + "aggregations":[ + { + "type": "HLLSketchMerge", + "name": "approxCountHLL", + "fieldName": "HLLSketchBuild", + "lgK": 12, + "tgtHllType": "HLL_4", + "round": true + }, + { + "type":"thetaSketch", + "name":"approxCountTheta", + "fieldName":"thetaSketch", + "size":16384, + "shouldFinalize":true, + "isInputThetaSketch":false, + "errorBoundsStdDev":null + }, + { + "type":"quantilesDoublesSketch", + "name":"quantilesSketch", + "fieldName":"quantilesDoublesSketch", + "k":128 + } + ] + }, + "expectedResults":[ + { + "timestamp" : "2013-01-01T00:00:00.000Z", + "result" : { + "quantilesSketch":10, + "approxCountTheta":5.0, + "approxCountHLL":5 + } + } + ] + }, + { + "description":"having spec on post aggregation", + "query":{ + "queryType":"groupBy", + "dataSource":"%%DATASOURCE%%", + "granularity":"day", + "dimensions":[ + "page" + ], + "filter":{ + "type":"selector", + "dimension":"language", + "value":"zh" + }, + "aggregations":[ + { + "type":"count", + "name":"rows" + }, + { + "type":"longSum", + "fieldName":"added", + "name":"added_count" + } + ], + "postAggregations": [ + { + "type":"arithmetic", + "name":"added_count_times_ten", + "fn":"*", + "fields":[ + {"type":"fieldAccess", "name":"added_count", "fieldName":"added_count"}, + {"type":"constant", "name":"const", "value":10} + ] + } + ], + "having":{"type":"greaterThan", "aggregation":"added_count_times_ten", "value":9000}, + "intervals":[ + "2013-01-01T00:00/2014-01-01T00:00" + ] + }, + "expectedResults":[ { + "version" : "v1", + "timestamp" : "2013-01-01T00:00:00.000Z", + "event" : { + "added_count_times_ten" : 18100.0, + "page" : "Crimson Typhoon", + "added_count" : 1810, + "rows" : 1 + } + } ] + }, + { + "description": "timeseries, stringFirst/stringLast aggs, all", + "query":{ + "queryType" : "timeseries", + "dataSource": "%%DATASOURCE%%", + "granularity":"day", + "intervals":[ + "2013-01-01T00:00/2014-01-01T00:00" + ], + "filter":null, + "aggregations":[ + { + "type": "stringFirst", + "name": "first_user", + "fieldName": "user" + }, + { + "type":"stringLast", + "name":"last_user", + "fieldName":"user" + } + ] + }, + "expectedResults":[ + { + "timestamp" : "2013-01-01T00:00:00.000Z", + "result" : { + "first_user":"masterYi", + "last_user":"speed" + } + } + ] + } +] \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_rollup_queries.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_rollup_queries.json new file mode 100644 index 00000000000..77854ea0acf --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_rollup_queries.json @@ -0,0 +1,56 @@ +[ + { + "description": "rows count", + "query":{ + "queryType" : "timeseries", + "dataSource": "%%DATASOURCE%%", + "granularity":"day", + "intervals":[ + "2013-08-31T00:00/2013-09-01T00:00" + ], + "filter": { + "type": "selector", + "dimension": "language", + "value": "en", + "extractionFn": null + }, + "aggregations":[ + { + "type": "count", + "name": "count" + } + ] + }, + "expectedResults":[ + { + "timestamp" : "2013-08-31T00:00:00.000Z", + "result" : { + "count":%%EXPECTED_COUNT_RESULT%% + } + } + ] + }, + { + "description": "scan with filter", + "query":{ + "queryType" : "scan", + "dataSource": "%%DATASOURCE%%", + "granularity":"day", + "intervals":[ + "2013-08-31T00:00/2013-09-01T00:00" + ], + "filter": { + "type": "selector", + "dimension": "language", + "value": "en", + "extractionFn": null + }, + "columns": [ + %%FIELD_TO_QUERY%% + ], + "resultFormat":"compactedList" + }, + "expectedResults": %%EXPECTED_SCAN_RESULT%%, + "fieldsToTest": ["events"] + } +] \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task.json new file mode 100644 index 00000000000..00bf7721f2b --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task.json @@ -0,0 +1,89 @@ +{ + "type": "index", + "spec": { + "dataSchema": { + "dataSource": "%%DATASOURCE%%", + "timestampSpec": { + "column": "timestamp", + "format": "auto" + }, + "dimensionsSpec": { + "dimensions": [ + "page", + {"type": "string", "name": "language", "createBitmapIndex": false}, + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city" + ] + }, + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + }, + { + "name": "thetaSketch", + "type": "thetaSketch", + "fieldName": "user" + }, + { + "name": "quantilesDoublesSketch", + "type": "quantilesDoublesSketch", + "fieldName": "delta" + }, + { + "name": "HLLSketchBuild", + "type": "HLLSketchBuild", + "fieldName": "user" + } + ], + "granularitySpec": { + "segmentGranularity": "DAY", + "queryGranularity": "second", + "intervals" : [ "2013-08-31/2013-09-02" ] + } + }, + "ioConfig": { + "type": "index", + "inputSource": { + "type": "local", + "files": [ + "/resources/data/batch_index/json/wikipedia_index_data1.json", + "/resources/data/batch_index/json/wikipedia_index_data2.json", + "/resources/data/batch_index/json/wikipedia_index_data3.json" + ] + }, + "inputFormat": { + "type": "json" + }, + "appendToExisting": false + }, + "tuningConfig": { + "type": "index", + "maxRowsPerSegment": 3, + "awaitSegmentAvailabilityTimeoutMillis": %%SEGMENT_AVAIL_TIMEOUT_MILLIS%% + } + } +} \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task_with_dimension_spec.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task_with_dimension_spec.json new file mode 100644 index 00000000000..1fa8b4eba32 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task_with_dimension_spec.json @@ -0,0 +1,86 @@ +{ + "type": "index", + "spec": { + "dataSchema": { + "dataSource": "%%DATASOURCE%%", + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + }, + { + "name": "thetaSketch", + "type": "thetaSketch", + "fieldName": "user" + }, + { + "name": "quantilesDoublesSketch", + "type": "quantilesDoublesSketch", + "fieldName": "delta" + }, + { + "name": "HLLSketchBuild", + "type": "HLLSketchBuild", + "fieldName": "user" + } + ], + "granularitySpec": { + "segmentGranularity": "DAY", + "queryGranularity": "DAY", + "intervals" : [ "2013-08-31/2013-09-02" ] + }, + "parser": { + "parseSpec": { + "format" : "json", + "timestampSpec": { + "column": "timestamp" + }, + "dimensionsSpec": { + "dimensions": [ + "page", + {"type": "string", "name": "language", "createBitmapIndex": false}, + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city" + ] + } + } + } + }, + "ioConfig": { + "type": "index", + "firehose": { + "type": "local", + "baseDir": "/resources/data/batch_index/json", + "filter": "wikipedia_index_data*" + } + }, + "tuningConfig": { + "type": "index", + "maxRowsPerSegment": 3, + "awaitSegmentAvailabilityTimeoutMillis": %%SEGMENT_AVAIL_TIMEOUT_MILLIS%% + } + } +} \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task_with_granularity_spec.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task_with_granularity_spec.json new file mode 100644 index 00000000000..544d191d925 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task_with_granularity_spec.json @@ -0,0 +1,71 @@ +{ + "type": "index", + "spec": { + "dataSchema": { + "dataSource": "%%DATASOURCE%%", + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + }, + { + "name": "thetaSketch", + "type": "thetaSketch", + "fieldName": "user" + }, + { + "name": "quantilesDoublesSketch", + "type": "quantilesDoublesSketch", + "fieldName": "delta" + }, + { + "name": "HLLSketchBuild", + "type": "HLLSketchBuild", + "fieldName": "user" + } + ], + "granularitySpec": %%GRANULARITYSPEC%%, + "parser": { + "parseSpec": { + "format" : "json", + "timestampSpec": { + "column": "timestamp" + }, + "dimensionsSpec": { + "dimensions": [ + {"type": "string", "name": "language", "createBitmapIndex": false} + ] + } + } + } + }, + "ioConfig": { + "type": "index", + "firehose": { + "type": "local", + "baseDir": "/resources/data/batch_index/json", + "filter": "wikipedia_index_data*" + } + }, + "tuningConfig": { + "type": "index", + "maxRowsPerSegment": 10, + "awaitSegmentAvailabilityTimeoutMillis": %%SEGMENT_AVAIL_TIMEOUT_MILLIS%% + } + } +} \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task_with_inputsource_transform.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task_with_inputsource_transform.json new file mode 100644 index 00000000000..e9bcbfd0aae --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task_with_inputsource_transform.json @@ -0,0 +1,103 @@ +{ + "type" : "index", + "spec" : { + "dataSchema" : { + "dataSource" : "%%DATASOURCE%%", + "timestampSpec": { + "column": "timestamp" + }, + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "triple-added", + "fieldName": "triple-added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + }, + { + "name": "thetaSketch", + "type": "thetaSketch", + "fieldName": "user" + }, + { + "name": "quantilesDoublesSketch", + "type": "quantilesDoublesSketch", + "fieldName": "delta" + }, + { + "name": "HLLSketchBuild", + "type": "HLLSketchBuild", + "fieldName": "user" + } + ], + "granularitySpec": { + "segmentGranularity": "DAY", + "queryGranularity": "second", + "intervals" : [ "2013-08-31/2013-09-02" ] + }, + "dimensionsSpec": { + "dimensions": [ + "page", + {"type": "string", "name": "language", "createBitmapIndex": false}, + "user", + "unpatrolled", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city" + ] + }, + "transformSpec": { + "transforms": [ + { + "type": "expression", + "name": "language", + "expression": "concat('language-', language)" + }, + { + "type": "expression", + "name": "triple-added", + "expression": "added * 3" + } + ] + } + }, + "ioConfig" : { + "type" : "index", + "inputSource" : { + "type" : "local", + "baseDir" : "/resources/data/batch_index/json", + "filter" : "wikipedia_index_data*" + }, + "inputFormat" : { + "type" : "json" + }, + "appendToExisting" : false + }, + "tuningConfig" : { + "type" : "index", + "maxRowsPerSegment" : 5000000, + "maxRowsInMemory" : 25000 + } + } +} diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task_with_transform.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task_with_transform.json new file mode 100644 index 00000000000..9f3128fb303 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task_with_transform.json @@ -0,0 +1,103 @@ +{ + "type": "index", + "spec": { + "dataSchema": { + "dataSource": "%%DATASOURCE%%", + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "triple-added", + "fieldName": "triple-added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + }, + { + "name": "thetaSketch", + "type": "thetaSketch", + "fieldName": "user" + }, + { + "name": "quantilesDoublesSketch", + "type": "quantilesDoublesSketch", + "fieldName": "delta" + }, + { + "name": "HLLSketchBuild", + "type": "HLLSketchBuild", + "fieldName": "user" + } + ], + "granularitySpec": { + "segmentGranularity": "DAY", + "queryGranularity": "second", + "intervals" : [ "2013-08-31/2013-09-02" ] + }, + "parser": { + "parseSpec": { + "format" : "json", + "timestampSpec": { + "column": "timestamp" + }, + "dimensionsSpec": { + "dimensions": [ + "page", + {"type": "string", "name": "language", "createBitmapIndex": false}, + "user", + "unpatrolled", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city" + ] + } + } + }, + "transformSpec": { + "transforms": [ + { + "type": "expression", + "name": "language", + "expression": "concat('language-', language)" + }, + { + "type": "expression", + "name": "triple-added", + "expression": "added * 3" + } + ] + } + }, + "ioConfig": { + "type": "index", + "firehose": { + "type": "local", + "baseDir": "/resources/data/batch_index/json", + "filter": "wikipedia_index_data*" + } + }, + "tuningConfig": { + "type": "index", + "maxRowsPerSegment": 3 + } + } +} \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_with_merge_column_limit_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_with_merge_column_limit_task.json new file mode 100644 index 00000000000..35b115c9f19 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_with_merge_column_limit_task.json @@ -0,0 +1,86 @@ +{ + "type": "index", + "spec": { + "dataSchema": { + "dataSource": "%%DATASOURCE%%", + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + }, + { + "name": "thetaSketch", + "type": "thetaSketch", + "fieldName": "user" + }, + { + "name": "quantilesDoublesSketch", + "type": "quantilesDoublesSketch", + "fieldName": "delta" + }, + { + "name": "HLLSketchBuild", + "type": "HLLSketchBuild", + "fieldName": "user" + } + ], + "granularitySpec": { + "segmentGranularity": "DAY", + "queryGranularity": "second", + "intervals" : [ "2013-08-31/2013-09-02" ] + }, + "parser": { + "parseSpec": { + "format" : "json", + "timestampSpec": { + "column": "timestamp" + }, + "dimensionsSpec": { + "dimensions": [ + "page", + {"type": "string", "name": "language", "createBitmapIndex": false}, + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city" + ] + } + } + } + }, + "ioConfig": { + "type": "index", + "firehose": { + "type": "local", + "baseDir": "/resources/data/batch_index/json", + "filter": "wikipedia_index_data*" + } + }, + "tuningConfig": { + "type": "index", + "maxRowsPerSegment": 3, + "maxColumnsToMerge" : 30 + } + } +} \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_local_input_source_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_local_input_source_index_task.json new file mode 100644 index 00000000000..ee0fd73021a --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_local_input_source_index_task.json @@ -0,0 +1,89 @@ +{ + "type": "index_parallel", + "spec": { + "dataSchema": { + "dataSource": "%%DATASOURCE%%", + "timestampSpec": { + "column": "timestamp" + }, + "dimensionsSpec": { + "dimensions": [ + "page", + {"type": "string", "name": "language", "createBitmapIndex": false}, + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city" + ] + }, + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + }, + { + "name": "thetaSketch", + "type": "thetaSketch", + "fieldName": "user" + }, + { + "name": "quantilesDoublesSketch", + "type": "quantilesDoublesSketch", + "fieldName": "delta" + }, + { + "name": "HLLSketchBuild", + "type": "HLLSketchBuild", + "fieldName": "user" + } + ], + "granularitySpec": { + "segmentGranularity": "DAY", + "queryGranularity": "second", + "intervals" : [ "2013-08-31/2013-09-02" ] + } + }, + "ioConfig": { + "type": "index_parallel", + "inputSource": { + "type": "local", + "filter" : "%%INPUT_SOURCE_FILTER%%", + "baseDir": "%%INPUT_SOURCE_BASE_DIR%%" + }, + "appendToExisting": %%APPEND_TO_EXISTING%%, + "dropExisting": %%DROP_EXISTING%%, + "inputFormat": %%INPUT_FORMAT%% + }, + "tuningConfig": { + "type": "index_parallel", + "maxNumConcurrentSubTasks": 4, + "splitHintSpec": { + "type": "maxSize", + "maxNumFiles": 1 + }, + "forceGuaranteedRollup": %%FORCE_GUARANTEED_ROLLUP%%, + "partitionsSpec": %%PARTITIONS_SPEC%% + } + } +} \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_merge_index_queries.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_merge_index_queries.json new file mode 100644 index 00000000000..ab4674999b5 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_merge_index_queries.json @@ -0,0 +1,42 @@ +[ + { + "description": "groupby, stringFirst/stringLast rollup aggs, all", + "query":{ + "queryType" : "groupBy", + "dataSource": "%%DATASOURCE%%", + "granularity":"day", + "dimensions":[ + "continent" + ], + "intervals":[ + "2013-08-31T00:00/2013-09-01T00:00" + ], + "filter":{ + "type":"selector", + "dimension":"continent", + "value":"Asia" + }, + "aggregations":[ + { + "type": "stringFirst", + "name": "earliest_user", + "fieldName": "first_user" + }, + { + "type":"stringLast", + "name":"latest_user", + "fieldName":"last_user" + } + ] + }, + "expectedResults":[ { + "version" : "v1", + "timestamp" : "2013-08-31T00:00:00.000Z", + "event" : { + "continent":"Asia", + "earliest_user":"masterYi", + "latest_user":"stringer" + } + } ] + } +] \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_merge_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_merge_index_task.json new file mode 100644 index 00000000000..43264a8c675 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_merge_index_task.json @@ -0,0 +1,70 @@ +{ + "type": "index", + "spec": { + "dataSchema": { + "dataSource": "%%DATASOURCE%%", + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + }, + { + "type": "stringFirst", + "name": "first_user", + "fieldName": "user" + }, + { + "type": "stringLast", + "name": "last_user", + "fieldName": "user" + } + ], + "granularitySpec": { + "segmentGranularity": "DAY", + "queryGranularity": "DAY", + "intervals" : [ "2013-08-31/2013-09-02" ] + }, + "parser": { + "parseSpec": { + "format" : "json", + "timestampSpec": { + "column": "timestamp" + }, + "dimensionsSpec": { + "dimensions": [ + "continent" + ] + } + } + } + }, + "ioConfig": { + "type": "index", + "firehose": { + "type": "local", + "baseDir": "/resources/data/batch_index/json", + "filter": "wikipedia_index_data*" + } + }, + "tuningConfig": { + "type": "index", + "maxRowsPerSegment": 5, + "maxRowsInMemory": 2 + } + } +} diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_merge_reindex_druid_input_source_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_merge_reindex_druid_input_source_task.json new file mode 100644 index 00000000000..9daae62c8d4 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_merge_reindex_druid_input_source_task.json @@ -0,0 +1,63 @@ +{ + "type": "index", + "spec": { + "ioConfig": { + "type": "index", + "inputSource": { + "type": "druid", + "dataSource": "%%DATASOURCE%%", + "interval": "2013-08-31/2013-09-01" + } + }, + "tuningConfig": { + "type": "index", + "partitionsSpec": { + "type": "dynamic" + } + }, + "dataSchema": { + "dataSource": "%%REINDEX_DATASOURCE%%", + "granularitySpec": { + "type": "uniform", + "queryGranularity": "DAY", + "segmentGranularity": "DAY" + }, + "timestampSpec": { + "column": "__time", + "format": "iso" + }, + "dimensionsSpec": { + "dimensions": [ + "continent" + ] + }, + "metricsSpec": [ + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + }, + { + "type": "stringFirst", + "name": "first_user", + "fieldName": "first_user" + }, + { + "type": "stringLast", + "name": "last_user", + "fieldName": "last_user" + } + ] + } + } +} \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_merge_reindex_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_merge_reindex_task.json new file mode 100644 index 00000000000..127461dd117 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_merge_reindex_task.json @@ -0,0 +1,65 @@ +{ + "type": "index", + "spec": { + "dataSchema": { + "dataSource": "%%REINDEX_DATASOURCE%%", + "metricsSpec": [ + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + }, + { + "type": "stringFirst", + "name": "first_user", + "fieldName": "first_user" + }, + { + "type": "stringLast", + "name": "last_user", + "fieldName": "last_user" + } + ], + "granularitySpec": { + "segmentGranularity": "DAY", + "queryGranularity": "DAY", + "intervals" : [ "2013-08-31/2013-09-01" ] + }, + "parser": { + "parseSpec": { + "format" : "json", + "timestampSpec": { + "column": "timestamp", + "format": "iso" + }, + "dimensionsSpec": { + "dimensions": [ + "continent" + ] + } + } + } + }, + "ioConfig": { + "type": "index", + "firehose": { + "type": "ingestSegment", + "dataSource": "%%DATASOURCE%%", + "interval": "2013-08-31/2013-09-01" + } + }, + "tuningConfig": { + "type": "index" + } + } +} diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_override_credentials_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_override_credentials_index_task.json new file mode 100644 index 00000000000..12b87977743 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_override_credentials_index_task.json @@ -0,0 +1,82 @@ +{ + "type": "index", + "spec": { + "dataSchema": { + "dataSource": "%%DATASOURCE%%", + "timestampSpec": { + "column": "timestamp" + }, + "dimensionsSpec": { + "dimensions": [ + "page", + {"type": "string", "name": "language", "createBitmapIndex": false}, + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city" + ] + }, + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + }, + { + "name": "thetaSketch", + "type": "thetaSketch", + "fieldName": "user" + }, + { + "name": "quantilesDoublesSketch", + "type": "quantilesDoublesSketch", + "fieldName": "delta" + }, + { + "name": "HLLSketchBuild", + "type": "HLLSketchBuild", + "fieldName": "user" + } + ], + "granularitySpec": { + "segmentGranularity": "DAY", + "queryGranularity": "second", + "intervals" : [ "2013-08-31/2013-09-02" ] + } + }, + "ioConfig": { + "type": "index", + "inputSource": { + "type": "%%INPUT_SOURCE_TYPE%%", + "properties": %%INPUT_SOURCE_CONFIG%%, + "%%INPUT_SOURCE_PROPERTY_KEY%%": %%INPUT_SOURCE_PROPERTY_VALUE%% + }, + "inputFormat": { + "type": "json" + } + }, + "tuningConfig": { + "type": "index" + } + } +} \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_druid_input_source_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_druid_input_source_index_task.json new file mode 100644 index 00000000000..46af17af598 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_druid_input_source_index_task.json @@ -0,0 +1,64 @@ +{ + "type": "index_parallel", + "spec": { + "dataSchema": { + "dataSource": "%%REINDEX_DATASOURCE%%", + "dimensionsSpec": { + "dimensionExclusions": [ + "robot", + "continent" + ] + }, + "timestampSpec": { + "column": "ignored-see-ignoreTimestampSpecForDruidInputSource", + "format": "iso" + }, + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + } + ], + "granularitySpec": { + "segmentGranularity": "DAY", + "queryGranularity": "second", + "intervals": [ + "2013-08-31/2013-09-02" + ] + } + }, + "ioConfig": { + "type": "index_parallel", + "inputSource": { + "type": "druid", + "dataSource": "%%DATASOURCE%%", + "interval": "2013-08-31/2013-09-02" + } + }, + "tuningConfig": { + "type": "index_parallel", + "maxNumConcurrentSubTasks": 10, + "forceGuaranteedRollup": "%%FORCE_GUARANTEED_ROLLUP%%", + "partitionsSpec": %%PARTITIONS_SPEC%%, + "splitHintSpec": { + "type":"segments", + "maxInputSegmentBytesPerTask": 1 + } + } + } +} diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_index_queries.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_index_queries.json new file mode 100644 index 00000000000..9618ba9e9b6 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_index_queries.json @@ -0,0 +1,71 @@ +[ + { + "description": "timeseries, 1 agg, all", + "query":{ + "queryType" : "timeBoundary", + "dataSource": "%%DATASOURCE%%" + }, + "expectedResults":[ + { + "timestamp" : "2013-08-31T01:02:33.000Z", + "result" : { + "minTime" : "2013-08-31T01:02:33.000Z", + "maxTime" : "2013-09-01T12:41:27.000Z" + } + } + ] + }, + + { + "description":"having spec on post aggregation", + "query":{ + "queryType":"groupBy", + "dataSource":"%%DATASOURCE%%", + "granularity":"day", + "dimensions":[ + "page" + ], + "filter":{ + "type":"selector", + "dimension":"language", + "value":"zh" + }, + "aggregations":[ + { + "type":"count", + "name":"rows" + }, + { + "type":"longSum", + "fieldName":"added", + "name":"added_count" + } + ], + "postAggregations": [ + { + "type":"arithmetic", + "name":"added_count_times_ten", + "fn":"*", + "fields":[ + {"type":"fieldAccess", "name":"added_count", "fieldName":"added_count"}, + {"type":"constant", "name":"const", "value":10} + ] + } + ], + "having":{"type":"greaterThan", "aggregation":"added_count_times_ten", "value":9000}, + "intervals":[ + "2013-08-31T00:00/2013-09-01T00:00" + ] + }, + "expectedResults":[ { + "version" : "v1", + "timestamp" : "2013-08-31T00:00:00.000Z", + "event" : { + "added_count_times_ten" : 9050.0, + "page" : "Crimson Typhoon", + "added_count" : 905, + "rows" : 1 + } + } ] + } +] \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_index_task.json new file mode 100644 index 00000000000..e83b1109da0 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_index_task.json @@ -0,0 +1,77 @@ +{ + "type": "index_parallel", + "spec": { + "dataSchema": { + "dataSource": "%%DATASOURCE%%", + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + } + ], + "granularitySpec": { + "segmentGranularity": "DAY", + "queryGranularity": "second", + "intervals" : [ "2013-08-31/2013-09-02" ] + }, + "parser": { + "parseSpec": { + "format" : "json", + "timestampSpec": { + "column": "timestamp" + }, + "dimensionsSpec": { + "dimensions": [ + "page", + {"type": "string", "name": "language", "createBitmapIndex": false}, + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city" + ] + } + } + } + }, + "ioConfig": { + "type": "index_parallel", + "firehose": { + "type": "local", + "baseDir": "/resources/data/batch_index/json", + "filter": "wikipedia_index_data*" + } + }, + "tuningConfig": { + "type": "index_parallel", + "maxNumConcurrentSubTasks": 10, + "forceGuaranteedRollup": "%%FORCE_GUARANTEED_ROLLUP%%", + "splitHintSpec": { + "type": "maxSize", + "maxSplitSize": 1 + }, + "partitionsSpec": %%PARTITIONS_SPEC%%, + "awaitSegmentAvailabilityTimeoutMillis": %%SEGMENT_AVAIL_TIMEOUT_MILLIS%% + } + } +} \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_index_using_sqlinputsource_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_index_using_sqlinputsource_task.json new file mode 100644 index 00000000000..b6b0d54c2aa --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_index_using_sqlinputsource_task.json @@ -0,0 +1,88 @@ +{ + "type": "index_parallel", + "spec": { + "dataSchema": { + "dataSource": "%%DATASOURCE%%", + "timestampSpec": { + "column": "timestamp" + }, + "dimensionsSpec": { + "dimensions": [ + "page", + {"type": "string", "name": "language", "createBitmapIndex": false}, + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city" + ] + }, + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + }, + { + "name": "thetaSketch", + "type": "thetaSketch", + "fieldName": "user" + }, + { + "name": "quantilesDoublesSketch", + "type": "quantilesDoublesSketch", + "fieldName": "delta" + }, + { + "name": "HLLSketchBuild", + "type": "HLLSketchBuild", + "fieldName": "user" + } + ], + "granularitySpec": { + "segmentGranularity": "DAY", + "queryGranularity": "second", + "intervals" : [ "2013-08-31/2013-09-02" ] + } + }, + "ioConfig": { + "type": "index_parallel", + "inputSource": { + "type": "sql", + "database": { + "type": "mysql", + "connectorConfig": { + "connectURI": "jdbc:mysql://druid-metadata-storage/sqlinputsource", + "user": "druid", + "password": "diurd" + } + }, + "sqls": %%SQL_QUERY%% + } + }, + "tuningConfig": { + "type": "index_parallel", + "maxNumConcurrentSubTasks": 10, + "partitionsSpec": %%PARTITIONS_SPEC%% + } + } +} \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_ingest_segment_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_ingest_segment_index_task.json new file mode 100644 index 00000000000..e995a0f299e --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_ingest_segment_index_task.json @@ -0,0 +1,69 @@ +{ + "type": "index_parallel", + "spec": { + "dataSchema": { + "dataSource": "%%REINDEX_DATASOURCE%%", + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + } + ], + "granularitySpec": { + "segmentGranularity": "DAY", + "queryGranularity": "second", + "intervals": [ + "2013-08-31/2013-09-02" + ] + }, + "parser": { + "parseSpec": { + "format": "json", + "timestampSpec": { + "column": "timestamp" + }, + "dimensionsSpec": { + "dimensionExclusions": [ + "robot", + "continent" + ] + } + } + } + }, + "ioConfig": { + "type": "index_parallel", + "firehose": { + "type": "ingestSegment", + "dataSource": "%%DATASOURCE%%", + "interval": "2013-08-31/2013-09-02", + "maxInputSegmentBytesPerTask": 1 + } + }, + "tuningConfig": { + "type": "index_parallel", + "maxNumConcurrentSubTasks": 10, + "forceGuaranteedRollup": "%%FORCE_GUARANTEED_ROLLUP%%", + "splitHintSpec": { + "type": "maxSize", + "maxNumFiles": 1 + }, + "partitionsSpec": %%PARTITIONS_SPEC%% + } + } +} diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_reindex_queries.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_reindex_queries.json new file mode 100644 index 00000000000..bbbeca9b58a --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_reindex_queries.json @@ -0,0 +1,18 @@ +[ + { + "description": "timeseries, 1 agg, all should only show data2", + "query":{ + "queryType" : "timeBoundary", + "dataSource": "%%DATASOURCE%%" + }, + "expectedResults":[ + { + "timestamp" : "2013-08-31T11:58:39.000Z", + "result" : { + "minTime" : "2013-08-31T11:58:39.000Z", + "maxTime" : "2013-09-01T01:02:33.000Z" + } + } + ] + } +] \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_reindex_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_reindex_task.json new file mode 100644 index 00000000000..9ab153c6862 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_reindex_task.json @@ -0,0 +1,75 @@ +{ + "type": "index_parallel", + "spec": { + "dataSchema": { + "dataSource": "%%DATASOURCE%%", + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + } + ], + "granularitySpec": { + "segmentGranularity": "DAY", + "queryGranularity": "second" + }, + "parser": { + "parseSpec": { + "format" : "json", + "timestampSpec": { + "column": "timestamp" + }, + "dimensionsSpec": { + "dimensions": [ + "page", + {"type": "string", "name": "language", "createBitmapIndex": false}, + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city" + ] + } + } + } + }, + "ioConfig": { + "type": "index_parallel", + "firehose": { + "type": "local", + "baseDir": "/resources/data/batch_index/json", + "filter": "wikipedia_index_data2*" + } + }, + "tuningConfig": { + "type": "index_parallel", + "maxNumConcurrentSubTasks": 10, + "forceGuaranteedRollup": "%%FORCE_GUARANTEED_ROLLUP%%", + "splitHintSpec": { + "type": "maxSize", + "maxNumFiles": 1 + }, + "partitionsSpec": %%PARTITIONS_SPEC%% + } + } +} \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_realtime_appenderator_index_queries.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_realtime_appenderator_index_queries.json new file mode 100644 index 00000000000..46d5ec4395a --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_realtime_appenderator_index_queries.json @@ -0,0 +1,87 @@ +[ + { + "description": "timeBoundary", + "query": { + "queryType":"timeBoundary", + "dataSource":"%%DATASOURCE%%" + }, + "expectedResults":[ + { + "timestamp":"%%TIMEBOUNDARY_RESPONSE_TIMESTAMP%%", + "result": { + "maxTime" : "%%TIMEBOUNDARY_RESPONSE_MAXTIME%%", + "minTime":"%%TIMEBOUNDARY_RESPONSE_MINTIME%%" + } + } + ] + }, + { + "description": "timeseries", + "query": { + "queryType": "timeseries", + "dataSource": "%%DATASOURCE%%", + "intervals": [ "%%TIMESERIES_QUERY_START%%/%%TIMESERIES_QUERY_END%%" ], + "granularity": "all", + "aggregations": [ + {"type": "longSum", "fieldName": "count", "name": "edit_count"}, + {"type": "doubleSum", "fieldName": "added", "name": "chars_added"} + ] + }, + "expectedResults": [ + { + "timestamp" : "%%TIMESERIES_RESPONSE_TIMESTAMP%%", + "result" : { + "chars_added" : 1642.0, + "edit_count" : 22 + } + } + ] + }, + { + "description":"having spec on post aggregation", + "query":{ + "queryType":"groupBy", + "dataSource":"%%DATASOURCE%%", + "granularity":"minute", + "dimensions":[ + "page" + ], + "aggregations":[ + { + "type":"count", + "name":"rows" + }, + { + "type":"longSum", + "fieldName":"added", + "name":"added_count" + } + ], + "postAggregations": [ + { + "type":"arithmetic", + "name":"added_count_times_ten", + "fn":"*", + "fields":[ + {"type":"fieldAccess", "name":"added_count", "fieldName":"added_count"}, + {"type":"constant", "name":"const", "value":10} + ] + } + ], + "having":{"type":"greaterThan", "aggregation":"added_count_times_ten", "value":9000}, + "intervals":[ + "%%POST_AG_REQUEST_START%%/%%POST_AG_REQUEST_END%%" + ] + }, + "expectedResults":[ { + "version" : "v1", + "timestamp" : "%%POST_AG_RESPONSE_TIMESTAMP%%", + "event" : { + "added_count_times_ten" : 9050.0, + "page" : "Crimson Typhoon", + "added_count" : 905, + "rows" : 1 + } + } ] + } +] diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_realtime_appenderator_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_realtime_appenderator_index_task.json new file mode 100644 index 00000000000..9e773609cb7 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_realtime_appenderator_index_task.json @@ -0,0 +1,94 @@ +{ + "type": "index_realtime_appenderator", + "spec": { + "dataSchema": { + "dataSource": "%%DATASOURCE%%", + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + } + ], + "granularitySpec": { + "segmentGranularity": "minute", + "queryGranularity": "second" + }, + "parser": { + "type": "map", + "parseSpec": { + "format": "tsv", + "columns": [ + "timestamp", + "page", + "language", + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city", + "added", + "deleted", + "delta" + ], + "timestampSpec": { + "column": "timestamp", + "format": "iso" + }, + "dimensionsSpec": { + "dimensions": [ + "page", + "language", + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city" + ] + } + } + } + }, + "ioConfig": { + "type": "realtime", + "firehose": { + "type": "timed", + "shutoffTime": "#SHUTOFFTIME", + "delegate": { + "type": "receiver", + "serviceName": "eventReceiverServiceName", + "bufferSize": 100000 + } + } + }, + "tuningConfig": { + "type": "realtime_appenderator", + "maxRowsInMemory": 1, + "intermediatePersistPeriod": "PT1M" + } + } +} diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_realtime_index_queries.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_realtime_index_queries.json new file mode 100644 index 00000000000..e5fe33b6b15 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_realtime_index_queries.json @@ -0,0 +1,147 @@ +[ + { + "description": "timeBoundary", + "query": { + "queryType":"timeBoundary", + "dataSource":"%%DATASOURCE%%" + }, + "expectedResults":[ + { + "timestamp":"%%TIMEBOUNDARY_RESPONSE_TIMESTAMP%%", + "result": { + "maxTime" : "%%TIMEBOUNDARY_RESPONSE_MAXTIME%%", + "minTime":"%%TIMEBOUNDARY_RESPONSE_MINTIME%%" + } + } + ] + }, + { + "description": "timeseries", + "query": { + "queryType": "timeseries", + "dataSource": "%%DATASOURCE%%", + "intervals": [ "%%TIMESERIES_QUERY_START%%/%%TIMESERIES_QUERY_END%%" ], + "granularity": "all", + "aggregations": [ + {"type": "longSum", "fieldName": "count", "name": "edit_count"}, + {"type": "doubleSum", "fieldName": "added", "name": "chars_added"} + ] + }, + "expectedResults": [ + { + "timestamp" : "%%TIMESERIES_RESPONSE_TIMESTAMP%%", + "result" : { + "chars_added" : 1595.0, + "edit_count" : 21 + } + } + ] + }, + { + "description":"having spec on post aggregation", + "query":{ + "queryType":"groupBy", + "dataSource":"%%DATASOURCE%%", + "granularity":"minute", + "dimensions":[ + "page" + ], + "aggregations":[ + { + "type":"count", + "name":"rows" + }, + { + "type":"longSum", + "fieldName":"added", + "name":"added_count" + } + ], + "postAggregations": [ + { + "type":"arithmetic", + "name":"added_count_times_ten", + "fn":"*", + "fields":[ + {"type":"fieldAccess", "name":"added_count", "fieldName":"added_count"}, + {"type":"constant", "name":"const", "value":10} + ] + } + ], + "having":{"type":"greaterThan", "aggregation":"added_count_times_ten", "value":9000}, + "intervals":[ + "%%POST_AG_REQUEST_START%%/%%POST_AG_REQUEST_END%%" + ] + }, + "expectedResults":[ { + "version" : "v1", + "timestamp" : "%%POST_AG_RESPONSE_TIMESTAMP%%", + "event" : { + "added_count_times_ten" : 9050.0, + "page" : "Crimson Typhoon", + "added_count" : 905, + "rows" : 1 + } + } ] + }, + { + "description": "topN, 1 agg, join to inline", + "query": { + "queryType": "topN", + "dataSource": { + "type": "join", + "left": "%%DATASOURCE%%", + "right": { + "type": "inline", + "columnNames": ["language", "lookupLanguage"], + "columnTypes": ["string", "string"], + "rows": [ + ["en", "inline join!"] + ] + }, + "rightPrefix": "j.", + "condition": "language == \"j.language\"", + "joinType": "LEFT" + }, + "intervals": [ "%%TIMESERIES_QUERY_START%%/%%TIMESERIES_QUERY_END%%" ], + "granularity": "all", + "virtualColumns": [ + { + "type": "expression", + "name": "lookupLanguage", + "expression": "nvl(\"j.lookupLanguage\", \"language\")", + "outputType": "string" + } + ], + "aggregations": [ + { + "type": "longSum", + "name": "count", + "fieldName": "count" + } + ], + "dimension": "lookupLanguage", + "metric": "count", + "threshold": 3 + }, + "expectedResults": [ + { + "timestamp": "%%TIMESERIES_RESPONSE_TIMESTAMP%%", + "result": [ + { + "lookupLanguage": "inline join!", + "count": 14 + }, + { + "lookupLanguage": "ja", + "count": 3 + }, + { + "lookupLanguage": "ru", + "count": 3 + } + ] + } + ] + } +] diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_realtime_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_realtime_index_task.json new file mode 100644 index 00000000000..5f48162c488 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_realtime_index_task.json @@ -0,0 +1,98 @@ +{ + "type": "index_realtime", + "spec": { + "dataSchema": { + "dataSource": "%%DATASOURCE%%", + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + } + ], + "granularitySpec": { + "segmentGranularity": "minute", + "queryGranularity": "second" + }, + "parser": { + "type": "map", + "parseSpec": { + "format": "tsv", + "columns": [ + "timestamp", + "page", + "language", + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city", + "added", + "deleted", + "delta" + ], + "timestampSpec": { + "column": "timestamp", + "format": "iso" + }, + "dimensionsSpec": { + "dimensions": [ + "page", + "language", + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city" + ] + } + } + } + }, + "ioConfig": { + "type": "realtime", + "firehose": { + "type": "timed", + "shutoffTime": "#SHUTOFFTIME", + "delegate": { + "type": "receiver", + "serviceName": "eventReceiverServiceName", + "bufferSize": 100000 + } + } + }, + "tuningConfig": { + "type": "realtime", + "maxRowsInMemory": 1, + "intermediatePersistPeriod": "PT1M", + "windowPeriod": "PT1M", + "rejectionPolicy": { + "type": "serverTime" + } + } + } +} diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_druid_input_source_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_druid_input_source_task.json new file mode 100644 index 00000000000..cf2415c2b45 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_druid_input_source_task.json @@ -0,0 +1,51 @@ +{ + "type": "index", + "spec": { + "ioConfig": { + "type": "index", + "inputSource": { + "type": "druid", + "dataSource": "%%DATASOURCE%%", + "interval": "2013-08-31/2013-09-01" + } + }, + "tuningConfig": { + "type": "index", + "partitionsSpec": { + "type": "dynamic" + } + }, + "dataSchema": { + "dataSource": "%%REINDEX_DATASOURCE%%", + "granularitySpec": { + "type": "uniform", + "queryGranularity": "SECOND", + "segmentGranularity": "DAY" + }, + "timestampSpec": { + "column": "__time", + "format": "millis" + }, + "dimensionsSpec": { + "dimensionExclusions" : ["robot", "continent"] + }, + "metricsSpec": [ + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + } + ] + } + } +} \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_druid_input_source_task_with_transforms.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_druid_input_source_task_with_transforms.json new file mode 100644 index 00000000000..2c2b0372a56 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_druid_input_source_task_with_transforms.json @@ -0,0 +1,106 @@ +{ + "type": "index", + "spec": { + "ioConfig": { + "type": "index", + "inputSource": { + "type": "druid", + "dataSource": "%%DATASOURCE%%", + "interval": "2013-08-31/2013-09-01" + } + }, + "tuningConfig": { + "type": "index", + "partitionsSpec": { + "type": "dynamic" + } + }, + "dataSchema": { + "dataSource": "%%REINDEX_DATASOURCE%%", + "granularitySpec": { + "type": "uniform", + "queryGranularity": "SECOND", + "segmentGranularity": "DAY" + }, + "timestampSpec": { + "column": "__time", + "format": "millis" + }, + "dimensionsSpec": { + "dimensions": [ + {"type": "string", "name": "language", "createBitmapIndex": false}, + "user", + "unpatrolled", + "page", + "newPage", + "anonymous", + "namespace", + "country", + "region", + "city" + ] + }, + "transformSpec": { + "transforms": [ + { + "type": "expression", + "name": "newPage", + "expression": "page" + }, + { + "type": "expression", + "name": "city", + "expression": "concat('city-', city)" + }, + { + "type": "expression", + "name": "one-plus-triple-added", + "expression": "\"triple-added\" + 1" + }, + { + "type": "expression", + "name": "delta", + "expression": "\"delta\" / 2" + }, + { + "type": "expression", + "name": "double-deleted", + "expression": "deleted * 2" + } + ] + }, + "metricsSpec": [ + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "triple-added", + "fieldName": "triple-added" + }, + { + "type": "doubleSum", + "name": "one-plus-triple-added", + "fieldName": "one-plus-triple-added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "double-deleted", + "fieldName": "double-deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + } + ] + } + } +} \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_queries.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_queries.json new file mode 100644 index 00000000000..572e0f1b5db --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_queries.json @@ -0,0 +1,66 @@ +[ + { + "description": "timeseries, 1 agg, all", + "query":{ + "queryType" : "timeBoundary", + "dataSource": "%%DATASOURCE%%" + }, + "expectedResults":[ + { + "timestamp" : "2013-08-31T01:02:33.000Z", + "result" : { + "minTime" : "2013-08-31T01:02:33.000Z", + "maxTime" : "2013-08-31T12:41:27.000Z" + } + } + ] + }, + + { + "description":"having spec on post aggregation", + "query":{ + "queryType":"groupBy", + "dataSource":"%%DATASOURCE%%", + "granularity":"day", + "dimensions":[ + "page" + ], + "filter":{ + "type":"selector", + "dimension":"language", + "value":"zh" + }, + "aggregations":[ + { + "type":"longSum", + "fieldName":"added", + "name":"added_count" + } + ], + "postAggregations": [ + { + "type":"arithmetic", + "name":"added_count_times_ten", + "fn":"*", + "fields":[ + {"type":"fieldAccess", "name":"added_count", "fieldName":"added_count"}, + {"type":"constant", "name":"const", "value":10} + ] + } + ], + "having":{"type":"greaterThan", "aggregation":"added_count_times_ten", "value":9000}, + "intervals":[ + "2013-08-31T00:00/2013-09-01T00:00" + ] + }, + "expectedResults":[ { + "version" : "v1", + "timestamp" : "2013-08-31T00:00:00.000Z", + "event" : { + "added_count_times_ten" : 9050.0, + "page" : "Crimson Typhoon", + "added_count" : 905 + } + } ] + } +] diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_queries_with_transforms.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_queries_with_transforms.json new file mode 100644 index 00000000000..40503121b92 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_queries_with_transforms.json @@ -0,0 +1,80 @@ +[ + { + "description": "timeseries, 1 agg, all", + "query":{ + "queryType" : "timeBoundary", + "dataSource": "%%DATASOURCE%%" + }, + "expectedResults":[ + { + "timestamp" : "2013-08-31T01:02:33.000Z", + "result" : { + "minTime" : "2013-08-31T01:02:33.000Z", + "maxTime" : "2013-08-31T12:41:27.000Z" + } + } + ] + }, + + { + "description":"having spec on post aggregation", + "query":{ + "queryType":"groupBy", + "dataSource":"%%DATASOURCE%%", + "granularity":"day", + "dimensions":[ + "newPage", + "city" + ], + "filter":{ + "type":"selector", + "dimension":"language", + "value":"language-zh" + }, + "aggregations":[ + { + "type":"longSum", + "fieldName":"one-plus-triple-added", + "name":"added_count" + }, + { + "type":"longSum", + "fieldName":"double-deleted", + "name":"double_deleted_count" + }, + { + "type":"longSum", + "fieldName":"delta", + "name":"delta_overshadowed" + } + ], + "postAggregations": [ + { + "type":"arithmetic", + "name":"added_count_times_ten", + "fn":"*", + "fields":[ + {"type":"fieldAccess", "name":"added_count", "fieldName":"added_count"}, + {"type":"constant", "name":"const", "value":10} + ] + } + ], + "having":{"type":"greaterThan", "aggregation":"added_count_times_ten", "value":9000}, + "intervals":[ + "2013-08-31T00:00/2013-09-01T00:00" + ] + }, + "expectedResults":[ { + "version" : "v1", + "timestamp" : "2013-08-31T00:00:00.000Z", + "event" : { + "added_count_times_ten" : 27160.0, + "newPage" : "Crimson Typhoon", + "city" : "city-Taiyuan", + "double_deleted_count" : 10, + "delta_overshadowed" : 450, + "added_count" : 2716 + } + } ] + } +] diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_task.json new file mode 100644 index 00000000000..e277a9127f4 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_task.json @@ -0,0 +1,53 @@ +{ + "type": "index", + "spec": { + "dataSchema": { + "dataSource": "%%REINDEX_DATASOURCE%%", + "metricsSpec": [ + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + } + ], + "granularitySpec": { + "segmentGranularity": "DAY", + "queryGranularity": "second", + "intervals" : [ "2013-08-31/2013-09-01" ] + }, + "parser": { + "parseSpec": { + "format" : "json", + "timestampSpec": { + "column": "timestamp", + "format": "iso" + }, + "dimensionsSpec": { + "dimensionExclusions" : ["robot", "continent"] + } + } + } + }, + "ioConfig": { + "type": "index", + "firehose": { + "type": "ingestSegment", + "dataSource": "%%DATASOURCE%%", + "interval": "2013-08-31/2013-09-01" + } + }, + "tuningConfig": { + "type": "index" + } + } +} diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_task_with_transforms.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_task_with_transforms.json new file mode 100644 index 00000000000..029b136d441 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_task_with_transforms.json @@ -0,0 +1,108 @@ +{ + "type": "index", + "spec": { + "dataSchema": { + "dataSource": "%%REINDEX_DATASOURCE%%", + "metricsSpec": [ + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "triple-added", + "fieldName": "triple-added" + }, + { + "type": "doubleSum", + "name": "one-plus-triple-added", + "fieldName": "one-plus-triple-added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "double-deleted", + "fieldName": "double-deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + } + ], + "granularitySpec": { + "segmentGranularity": "DAY", + "queryGranularity": "second", + "intervals" : [ "2013-08-31/2013-09-01" ] + }, + "parser": { + "parseSpec": { + "format" : "json", + "timestampSpec": { + "column": "timestamp", + "format": "iso" + }, + "dimensionsSpec": { + "dimensions": [ + {"type": "string", "name": "language", "createBitmapIndex": false}, + "user", + "unpatrolled", + "page", + "newPage", + "anonymous", + "namespace", + "country", + "region", + "city" + ] + }, + "transformSpec": { + "transforms": [ + { + "type": "expression", + "name": "newPage", + "expression": "page" + }, + { + "type": "expression", + "name": "city", + "expression": "concat('city-', city)" + }, + { + "type": "expression", + "name": "one-plus-triple-added", + "expression": "\"triple-added\" + 1" + }, + { + "type": "expression", + "name": "delta", + "expression": "\"delta\" / 2" + }, + { + "type": "expression", + "name": "double-deleted", + "expression": "deleted * 2" + } + ] + } + } + } + }, + "ioConfig": { + "type": "index", + "firehose": { + "type": "ingestSegment", + "dataSource": "%%DATASOURCE%%", + "interval": "2013-08-31/2013-09-01" + } + }, + "tuningConfig": { + "type": "index" + } + } +} diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_union_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_union_index_task.json new file mode 100644 index 00000000000..75c1281fcd2 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_union_index_task.json @@ -0,0 +1,98 @@ +{ + "type": "index_realtime", + "spec": { + "dataSchema": { + "dataSource": "%%DATASOURCE%%", + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + } + ], + "granularitySpec": { + "segmentGranularity": "DAY", + "queryGranularity": "second" + }, + "parser": { + "type": "map", + "parseSpec": { + "format": "tsv", + "columns": [ + "timestamp", + "page", + "language", + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city", + "added", + "deleted", + "delta" + ], + "timestampSpec": { + "column": "timestamp", + "format": "iso" + }, + "dimensionsSpec": { + "dimensions": [ + "page", + "language", + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city" + ] + } + } + } + }, + "ioConfig": { + "type": "realtime", + "firehose": { + "type": "timed", + "shutoffTime": "#SHUTOFFTIME", + "delegate": { + "type": "receiver", + "serviceName": "eventReceiverServiceName", + "bufferSize": 100000 + } + } + }, + "tuningConfig": { + "type": "realtime", + "maxRowsInMemory": 1, + "intermediatePersistPeriod": "PT1M", + "windowPeriod": "PT1M", + "rejectionPolicy": { + "type": "none" + } + } + } +} diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_with_timestamp_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_with_timestamp_index_task.json new file mode 100644 index 00000000000..4f13b70cb72 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_with_timestamp_index_task.json @@ -0,0 +1,86 @@ +{ + "type": "index", + "spec": { + "dataSchema": { + "dataSource": "%%DATASOURCE%%", + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + }, + { + "name": "thetaSketch", + "type": "thetaSketch", + "fieldName": "user" + }, + { + "name": "quantilesDoublesSketch", + "type": "quantilesDoublesSketch", + "fieldName": "delta" + }, + { + "name": "HLLSketchBuild", + "type": "HLLSketchBuild", + "fieldName": "user" + } + ], + "granularitySpec": { + "segmentGranularity": "DAY", + "queryGranularity": "second", + "intervals" : [ "2013-08-31/2013-09-02" ] + }, + "parser": { + "parseSpec": { + "format" : "json", + "timestampSpec": { + "column": "timestamp" + }, + "dimensionsSpec": { + "dimensions": [ + "page", + {"type": "string", "name": "language", "createBitmapIndex": false}, + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city", + "timestamp" + ] + } + } + } + }, + "ioConfig": { + "type": "index", + "firehose": { + "type": "local", + "baseDir": "/resources/data/batch_index/json", + "filter": "wikipedia_index_data*" + } + }, + "tuningConfig": { + "type": "index", + "maxRowsPerSegment": 3 + } + } +} \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikiticker_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikiticker_index_task.json new file mode 100644 index 00000000000..d450c7b9458 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikiticker_index_task.json @@ -0,0 +1,66 @@ +{ + "type": "index", + "spec": { + "dataSchema": { + "dataSource": "%%DATASOURCE%%", + "granularitySpec": { + "type": "uniform", + "segmentGranularity": "day", + "queryGranularity": "none", + "intervals": [ + "2015-09-12/2015-09-13" + ] + }, + "parser": { + "type": "hadoopyString", + "parseSpec": { + "format": "json", + "dimensionsSpec": { + "dimensions": [ + "channel", + "page", + "user" + ] + }, + "timestampSpec": { + "format": "auto", + "column": "time" + } + } + }, + "metricsSpec": [ + { + "name": "count", + "type": "count" + }, + { + "name": "added", + "type": "longSum", + "fieldName": "added" + }, + { + "name": "deleted", + "type": "longSum", + "fieldName": "deleted" + }, + { + "name": "delta", + "type": "longSum", + "fieldName": "delta" + } + ] + }, + "ioConfig": { + "type": "index", + "firehose": { + "type": "local", + "baseDir": "/shared/wikiticker-it", + "filter": "wikiticker-2015-09-12-sampled.json.gz" + } + }, + "tuningConfig": { + "type": "index", + "targetPartitionSize" : 10000 + } + } +} \ No newline at end of file diff --git a/integration-tests-ex/check-results.sh b/integration-tests-ex/check-results.sh deleted file mode 100755 index 44aac0aa9f6..00000000000 --- a/integration-tests-ex/check-results.sh +++ /dev/null @@ -1,89 +0,0 @@ -#! /bin/bash - -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -#-------------------------------------------------------------------- - -# Run from Travis which has no good way to attach logs to a -# build. Instead, we check if any IT failed. If so, we append -# the last 100 lines of each server log to stdout. We have to -# stay wihtin the 4MB limit which Travis applies, so we only -# emit logs for the first failure, and only for servers that -# don't report normal completion. -# -# The only good way to check for test failures is to parse -# the Failsafe summary for each test located in -# /target/failsafe-reports/failsafe-summary.xml -# -# This directory has many subdirectories, some of which are -# tests. We rely on the fact that a test starts with "it-" AND -# contains a failsafe report. (Some projects start with "it-" -# but are not tests.) - -# Run in the docker-tests directory -cd $(dirname $0) - -# Scan for candidate projects -for PROJECT in it-* -do - # Check if a failsafe report exists. It will exist if the directory is - # a test project and failsafe ran on that directory. - REPORTS="$PROJECT/target/failsafe-reports/failsafe-summary.xml" - if [ -f "$REPORTS" ] - then - # OK, so Bash isn't the world's best text processing language... - ERRS=1 - FAILS=1 - while IFS= read -r line - do - if [ "$line" = " 0" ] - then - ERRS=0 - fi - if [ "$line" = " 0" ] - then - FAILS=0 - fi - done < "$REPORTS" - if [ $ERRS -eq 1 -o $FAILS -eq 1 ] - then - FOUND_LOGS=0 - echo "======= $PROJECT Failed ==========" - # All logs except zookeeper - for log in $(ls $PROJECT/target/shared/logs/[a-y]*.log) - do - # We assume that a successful exit includes a line with the - # following: - # Stopping lifecycle [module] stage [INIT] - tail -5 "$log" | grep -Fq 'Stopping lifecycle [module] stage [INIT]' - if [ $? -ne 0 ] - then - # Assume failure and report tail - echo $(basename $log) "logtail ========================" - tail -100 "$log" - FOUND_LOGS=1 - fi - done - - # Only emit the first failure to avoid output bloat - if [ $FOUND_LOGS -eq 1 ] - then - exit 0 - else - echo "All Druid services exited normally." - fi - fi - fi -done diff --git a/integration-tests-ex/docs/compose.md b/integration-tests-ex/docs/compose.md new file mode 100644 index 00000000000..01b41896adb --- /dev/null +++ b/integration-tests-ex/docs/compose.md @@ -0,0 +1,238 @@ + + +# Docker Compose Configuration + +The integration tests use Docker Compose to launch Druid clusters. Each +test defines its own cluster +depending on what is to be tested. Since a large amount of the definition is +common, we use inheritance to simplify cluster definition. + +Tests are split into categories so that they can run in parallel. Some of +these categories use the same cluster configuration. To further reduce +redundancy, test categories can share cluster configurations. + +See also: + +* [Druid configuration](druid-config.md) which is done via Compose. +* [Test configuration](test-config.md) which tells tests about the + cluster configuration. +* [Docker compose specification](https://github.com/compose-spec/compose-spec/blob/master/spec.md) + +## File Structure + +Docker Compose files live in the `druid-it-cases` module (`test-cases` folder) +in the `cluster` directory. There is a separate subdirectory for each cluster type +(subset of test categories), plus a `Common` folder for shared files. + +## Shared Directory + +Each test has a "shared" directory that is mounted into each container to hold things +like logs, security files, etc. The directory is known as `/shared` within the container, +and resides in `target/`. Even if two categories share a cluster configuration, +they will have separate local versions of the shared directory. This is important to +keep log files separate for each category. + +## Base Configurations + +Test clusters run some number of third-party "infrastructure" containers, +and some number of Druid service containers. For the most part, each of +these services (in Compose terms) is similar from test to test. Compose +provides [an inheritance feature]( +https://github.com/compose-spec/compose-spec/blob/master/spec.md#extends) +that we use to define base configurations. + +* `cluster/Common/dependencies.yaml` defines external dependencis (MySQL, Kafka, ZK + etc.) +* `cluster/Common/druid.yaml` defines typical settings for each Druid service. + +Test-specific configurations extend and customize the above. + +### Druid Configuration + +Docker compose passes information to Docker in the form of environment variables. +The test use a variation of the environment-variable-based configuration used in +the [public Docker image](https://druid.apache.org/docs/latest/tutorials/docker.html). +That is, variables of the form `druid_my_config` are converted, by the image launch +script, into properties of the form `my.config`. These properties are then written +to a launch-specific `runtime.properties` file. + +Rather than have a test version of `runtime.properties`, instead we have a set of +files that define properties as environment variables. All are located in +`cases/cluster/Common/environment-configs`: + +* `common.env` - Properties common to all services. This is the test equivalent to + the `common.runtime.properties` file. +* `.env` - Properties unique to one service. This is the test equivalent to + the `service/runtime.properties` files. + +### Special Environment Variables + +Druid properties can be a bit awkward and verbose in a test environment. A number of +test-specific properties help: + +* `druid_standard_loadList` - Common extension load list for all tests, in the form + of a comma-delimited list of extensions (without the brackets.) Defined in + `common.env`. +* `druid_test_loadList` - A list of additional extensions to load for a specific test. + Defined in the `docker-compose.yaml` file for that test category. Do not include + quotes. + +Example test-specific list: + +```text +druid_test_loadList=druid-azure-extensions,my-extension +``` + +The launch script combines the two lists, and adds the required brackets and quotes. + +## Test-Specific Cluster + +Each test has a directory named `cluster/`. Docker Compose uses this name +as the cluster name which appears in the Docker desktop UI. The folder contains +the `docker-compose.yaml` file that defines the test cluster. + +In the simplest case, the file just lists the services to run as extensions +of the base services: + +```text +services: + zookeeper: + extends: + file: ../Common/dependencies.yaml + service: zookeeper + + broker: + extends: + file: ../Common/compose/druid.yaml + service: broker +... +``` + +## Cluster Configuration + +If a test wants to run two of some service (say Coordinator), then it +can use the "standard" definition for only one of them and must fill in +the details (especially distinct port numbers) for the second. +(See `HighAvilability` for an example.) + +By default, the container and internal host name is the same as the service +name. Thus, a `broker` service resides in a `broker` container known as +host `broker` on the Docker overlay network. +The service name is also usually the log file name. Thus `broker` logs +to `/target//logs/broker.log`. + +An environment variable `DRUID_INSTANCE` adds a suffix to the service +name and causes the log file to be `broker-one.log` if the instance +is `one`. The service name should have the full name `broker-one`. + +Druid configuration comes from the common and service-specific environment +files in `/compose/environment-config`. A test-specific service configuration +can override any of these settings using the `environment` section. +(See [Druid Configuration](druid-config.md) for details.) +For special cases, the service can define its configuration in-line and +not load the standard settings at all. + +Each service can override the Java options. However, in practice, the +only options that actually change are those for memory. As a result, +the memory settings reside in `DRUID_SERVICE_JAVA_OPTS`, which you can +easily change on a service-by-service or test-by-test basis. + +Debugging is enabled on port 8000 in the container. Each service that +wishes to expose debugging must map that container port to a distinct +host port. + +The easiest way understand the above is to look at a few examples. + +## Service Names + +The Docker Compose file sets up an "overlay" network to connect the containers. +Each is known via a host name taken from the service name. Thus "zookeeper" is +the name of the ZK service and of the container that runs ZK. Use these names +in configuration within each container. + +### Host Ports + +Outside of the application network, containers are accessible only via the +host ports defined in the Docker Compose files. Thus, ZK is known as `localhost:2181` +to tests and other code running outside of Docker. + +## Test-Specific Configuration + +In addition to the Druid configuration discussed above, the framework provides +three ways to pass test-specific configuration to the tests. All of these methods +override any configuration in the `docker-compose` or cluster `env` files. + +The values here are passed into the Druid server as configuration values. The +values apply to all services. (This mechanism does not allow service-specific +values.) In all three approaches, use the `druid_` environment variable form. + +Precendence is in the order below with the user file lowest priority and environment +variables highest. + +### User-specific `~/druid-it/ + +# Test Conversion + +Here is the process to convert an existing `integration-test` +group to this new structure. + +The tests all go into the `druid-integration-test-cases` module +(sub-directory `test-cases`). Move the tests into the existing +`testsEx` name space so they do not collide with the existing +integration test namespace. + +## Cluster Definition + +Define a category for your tests. See [tests](tests.md) for the +details. The category is the name of the cluster definition by +default. + +Determine if you can use an existing cluster definition, or if you +need to define a new one. See [tests](tests.md) for how to share a +cluster definition. If you share a definition, update `cluster.sh` +to map from your category name to the shared cluster definition +name. + +To create a new defnition, +create a `druid-cluster/docker-compose.yaml` file by converting the +previous `docker/docker-compose-.yml` file. Carefully review +each service. Use existing files as a guide. + +In `integration-tests` all groups share a set of files with many +conditionals to work out what is to be done. In this system, each +test group stands alone: its Docker Compose file defines the cluster +for that one test. There is some detangling of the existing conditionals +to determine the setup used by each test group. + +Create the `yaml/docker.yaml` resource in `/src/test/resources` to +define your cluster for the Java tests. + +Determine if the test group populates the metadata store using queries +run in the Docker container. If so, copy those queries into the +`docker.yaml` file in the `metadataInit` section. (In the new structure, +these setup queries run in the test client, not in each Docker service.) +See the former `druid.sh` script to see what SQL was used previously. + +### + +## Test Runner + +ITs require a large amount of setup. All that code is encapsulated in the +`DruidTestRunner` class: + +```java +@RunWith(DruidTestRunner.class) +@Category(MyCategory.class) +public class ITMyTest +``` + +It is helpful to know what the test runner does: + +* Loads the cluster configuration from the `docker.yaml` file, and + resolves any includes. +* Builds up the set of Guice modules needed for the test. +* Creates the Guice injector. +* Uses the injector to inject dependencies into your test class. +* Starts the Druid lifecycle. +* Waits for each Druid service defined in `docker.yaml` to become + available. +* Runs your test methods. +* Ends the Druid lifecycle. + +You can customize the configuration for non-standard cases. See +[tests](tests.md) for details. + +## Tests + +Convert the individual tests. + +### Basics + +Copy the existing tests for the target group into the +`druid-it-cases`. For sanity, you may want to do one by one. + +When adding tests, leave the original tests in `integration-tests` for +now. (Until we have the new system running in Travis.) Once Travis +runs, you can move, rather than copy, the tests. + +While we are copying, copy to the `org.apache.druid.testsEx` package to +prevent name conficts with `org.apache.druid.tests`. + +### Maven Dependencies + +You may need to add dependencies to `pom.xml`. + +The `docker-tests/pom.xml` file includes Maven dependencies for the most +common Druid modules, which transitiviely include the third-party modules +which the Druid modules reference. You test sub-project may need addition +dependencies. To find them, review `integration-tests/pom.xml`. Careful, +however, as that file is a bit of a "kitchen sink" that includes every +possible dependency, even those already available transitively. + +If you feel the dependency is one used by multiple tests, go ahead and +add it to `docker-tests/pom.xml`. If, however, it is somehwat unique to +the test group, just add it to that sub-modules `pom.xml` file instead. + +Use the following to verify the `pom.xml`: + +```bash +mvn dependency:analyze -DoutputXML=true -DignoreNonCompile=true \ + -P skip-static-checks -Ddruid.console.skip=true -Dmaven.javadoc.skip=true \ + -P skip-tests +``` + +Doing it now will save build cycles when submitting your PR. + +### Resources and Test Data + +The existing tests use the `/src/test/resources/` directory to hold both +JSON specs used by the tests, as well as test data used by the cluster. +To make the data available to tests, we mount the `/src/test/resources` +folder into the Indexer at `/resources`. + +In the new version, we separate these two groups of files. Those used by +tests continue to reside in `/src/test/resources` for the individual +tests. Those shared by multiple tests can be in `base-test/src/test/resources`. +Copy the resource files from `integration-tests` into one of these +locations. Try to avoid doing a bulk copy: copy only the files used by +the particular test group being converted. + +Then, copy the data into `/data`, keeping the same path. See +`data/README.md` for more information. + +#### To Do + +It may turn out that data files are shared among tests. In that case, we'd +want to put them in a common location, keeping test-specific data in the +project for that test. But, we can't easily combine two directories into +a single volume mount. + +Instead, we can use the `target/shared` folder: create a new `data` +folder, copy in the required files, and mount that at `/resources`. +Or, if we feel energetic, just change the specs to read their data +from `/shared/data`, since `/shared` is already mounted. + +### Extensions + +You may see build or other code that passes a list of extensions to an old +integration test. Such configuration represents a misunderstanding of how tests (as +clients) actually work. Tests nave no visibility to a Druid installation directory. +As a result, the "extension" concept does not apply. Instead, tests are run from +Maven, and are subject to the usual Maven process for locating jar files. That +means that any extensions which the test wants to use should be listed as dependencies +in the `pom.xml` file, and will be available on the class path. There is no need for, +or use of, the `druid_extensions_loadList` for tests (or, indeed, for any client.) + +### Starter Test (Optional) + +An optional step is to ease into the new system by doing a simple +"starter test". +Create a ad-hoc test file, say `StarterTest` to hold one of the +tests to be converted. Copy any needed Guice injections. This will +be a JUnit test. + +Define your test class like this: + +```java +@RunWith(DruidTestRunner.class) +public class StarterTest +``` + +The test runner handles the required startup, Guice configuration, cluster +validation, and shutdown. Just add your own test cases. + +Determine if the test runs queries from `src/test/resources/queries`. +If so, copy those to the new sub-project. Do the same with any other +resources which the test requires. + +In this new structure, each group is its own sub-project, so resources +are separted out per test group (sub-project), whereas in +`integration-tests` the resources are all grouped together. If there +are shared resources, put those in the `docker-tests/src/test/resources` +folder so they can be shared. (This may require creating a test-jar. +As an alternative, they can be put in `base-test` which is already +available to all tests.) + +Run the one test. This will find bugs in the above. It will also likely +point out that you need Druid modules not in the base set defined by +`Initialization`. Add these modules via the `Builder.modules()` method. +Resolve the other issues which will inevitably appear. + +This starter test will ensure that most of the dependency and configuration +issues are resolved. + +### Revised Helper Classes + +The new test structure adopted shorter container and host names: +`coordinator` instead of `druid-coordinator` etc. This is safe because the +Docker application runs in isolation, we don't have to worry about a +potential `coordinator` from application X. + +To handle these changes, there are new versions of several helper classes. +Modify the tests to use the new versions: + +* `DruidClusterAdminClient` - interfaces with Docker using hard-coded + container names. + +The old versions are in `org.apache.druid.testing.utils` in +`integration-tests`, the new versions in `org.apache.druid.testing2.utils` +in this project. + +### Test Classes + +You can now convert the bulk of the tests. +One-by-one, convert existing classes: + +* Remove the TestNG annotations and includes. Substitute JUnit includes. +* Add the `@RunWith` annotation. +* Run the test in the debugger to ensure it works. + +The test class definition should look like this: + +```java +@RunWith(DruidTestRunner.class) +public class ITIndexerTest ... +{ +``` + +Run the entire suite from Maven in the sub-module directory. It should +start the cluster, run the tests, and shut down the cluster. + +## Improving Tests + +Once the tests work, an optional step is to improvement a bit beyond what +was already done. + +### Retries + +The `Initializer` takes upon itself the task of ensuring that all services +are up (at least enough that they each report that they are healthy.) So, +it is not necessary for each test case to retry endlessly to handle the case +that it is the first test run on a cluster still coming up. We can remove +retries that don't represent valid server behavior. For example, if the goal +is too ensure that the endpoint `/foo` returns `bar`, then there is no need +to retry: if the server is up, then its `/foo` endpoint should be working, and +so it should return `bar`, assuming that the server is deterministic. + +If `bar` represents something that takes time to compute (the result of a +task, say), then retry is valid. If `bar` is deterministic, then retrying won't +fix a bug that causes `bar` to be reported incorrectly. + +Use your judgement to determine when retries were added "just to be safe" +(and can thus be removed), and when the represent actual race conditions in +the operation under tests. + +### Cluster Client + +The tests obviously do a large number of API calls to the server. Some (most) +seem to spell out the code inline, resulting in much copy/paste. An improvement +is to use the cluster client instead: `ClusterClient`. Add methods for endpoints +not yet covered by copying the code from the test in question. (Better, refactor +that code to use the existing lower-level `get()` and similar methods. Then, +use the cluster client method in place of the copy/paste wad of code. + +The result is a test that is smaller, easier to undestand, easier to maintain, +and easier debug. Also, future tests are easier to write because they can reuse +the method you added to the cluster client. + +You can inject the cluster client into your test: + +```java + @Inject + private ClusterClient clusterClient; +``` + +You may find that by using the cluster client, some of the dependencies which +the test needed are now unused. Go ahead and remove them. diff --git a/integration-tests-ex/docs/debugging.md b/integration-tests-ex/docs/debugging.md new file mode 100644 index 00000000000..fb0da9aee37 --- /dev/null +++ b/integration-tests-ex/docs/debugging.md @@ -0,0 +1,159 @@ + + +# Debugging the Druid Image and Integration Tests + +The integration setup has as a primary goal the ability to quickly debug +the Druid image and any individual tests. A first step is to move the +image build into a separate project. A second step is to ensure each +test can run in JUnit in an IDE against a cluster you start by hand. + +This section discusses how to use the various debugging features. + +See: + +* [Docker Configuration](docker.md) for information on debugging + docker builds. + +## General Debug Process + +Ease of debugging is a key goal of the revised structure. + +* Rebuild the Docker image only when the Druid code changes. + Do a normal distribution build, then build a new image. +* Reuse the same image over and over if you only change tests + (such as when adding a new test.) +* Reuse the same `shared` directory when the test does not + make permanent changes. +* Change Druid configuration by changing the Docker compose + files, no need to rebuild the image. +* Work primarily in the IDE when debugging tests. +* To add more logging, change the `log4j2.xml` file in the shared + directory to increase the logging level. +* Remote debug Druid services if needed. + +## Exploring the Test Cluster + +When run in Docker Compose, the endpoints known to Druid nodes differ from +those needed by a client sitting outside the cluster. We could provide an +explicit mapping. Better is to use the +[Router](https://druid.apache.org/docs/latest/design/router.html#router-as-management-proxy) +to proxy requests. Fortunately, the Druid Console already does this. + +## Docker Build Output + +Modern Docker seems to hide the output of commands, which is a hassle to debug +a build. Oddly, the details appear for a failed build, but not for success. +Use the followig to see at least some output: + +```bash +export DOCKER_BUILDKIT=0 +``` + +Once the base container is built, you can run it, log in and poke around. First +identify the name. See the last line of the container build: + +```text +Successfully tagged org.apache.druid/test: +``` + +Or ask Docker: + +```bash +docker images +``` + +## Debug the Docker Image + +You can log into the Docker image and poke around to see what's what: + + +```bash +docker run --rm -it --entrypoint bash org.apache.druid/test: +``` + +Quite a few environment variables are provided by Docker and the setup scripts +to see them, within the container, use: + +```bash +env +``` + +## Debug an Integration Test + +To debug an integration test, you need a Docker image with the latest Druid. +To get that, you need a full Druid build. So, we break the debugging process +down into steps that depend on the state of your code. Assume `DRUID_DEV` +points to your Druid development area. + +### On Each Druid Build + +If you need to rebuild Druid (because you fixed something), do: + +* Do a distribution build of Druid: +* Build the test image. + +See [quickstart](quickstart.md) for the commands. + +### Start the Test Cluster + +* Pick a test "group" to use. +* Start a test cluster configured for this test. +* Run a test from the command line: + +Again, see [quickstart](quickstart.md) for the commands. + +### Debug the Test + +To run from your IDE, find the test to run and run it as a JUnit test (with the +cluster up.) + +Depending on the test, you may be able to run the test over and over against the +same cluster. (In fact, you should try to design your tests so that this is true: +clean up after each run.) + +The tests are just plain old JUnit tests that happen to reach out to the +test cluster and/or Docker to do their work. You can set breakpoints and debug +in the usual way. + +Each test will first verify that the cluster is fully up before it starts, so +you can launch your debug session immediately after starting the cluster: the tests +will wait as needed. + +### Stop the Test Cluster + +When done, stop the cluster: [quickstart](quickstart.md) again for details. + +## Typical Issues + +For the most part, you can stop and restart the Druid services as often +as you like and Druid will just work. There are a few combinations that +can lead to trouble, however. + +* Services won't start: When doing a new build, stop the existing cluster + before doing the build. The build removes and rebuilds the shared + directory: services can't survive that. +* Metastore failure: The metastore container will recreate the DB on + each restart. This will fail if your shared directory already contains + a DB. Do a `rm -r target//db` before restarting the DB container. +* Coordinator fails with DB errors. The Coordinator will create the Druid + tables when it starts. This means the DB has to be created. If the DB + is removed after the Coordinator starts (to fix the above issue, say) + then you have to restart the Coordinator so it can create the needed + tables. diff --git a/integration-tests-ex/docs/dependencies.md b/integration-tests-ex/docs/dependencies.md new file mode 100644 index 00000000000..90047449f5c --- /dev/null +++ b/integration-tests-ex/docs/dependencies.md @@ -0,0 +1,239 @@ + + +# Dependencies + +The Docker tests have a number of dependencies which are important to understand +when making changes or debugging problems. + +## Third-Party Libraries + +As described in the [Docker](docker.md) section, the Docker image contains Druid +plus three external dependencies: + +* The MySQL client library +* The MariaDB client library +* The Kafka protobuf provider + +These libraries are not shipped with Druid itself. Instead, we add them to the +image as follows: + +* Dependencies are listed in the `test-image/pom.xml` file. +* Maven fetches the dependencides from an upstream repo and places them + into the local Maven cache. +* The `test-image/pom.xml` file uses the `maven-dependency-plugin` + to copy these dependencies from the local repo into the + `target/docker` directory. +* The `Dockerfile` copies the dependencies into the `/usr/local/druid/lib` + directory after `build-image.sh` has unpacked the Druid distribution + into `/usr/local/druid`. + +The key benefit is that the dependencies are downloaded once and are +served from the local repo afterwards. + +## Third-Party Servers + +As described in the [Docker](docker.md) section, we use third-party +"official" images for three of our external server dependencies: + +* [MySQL](https://hub.docker.com/_/mysql). This image is configured + to create the Druid database and user upon startup. +* [ZooKeeper](https://hub.docker.com/_/zookeeper). +* [Kafka](https://hub.docker.com/r/bitnami/kafka/). There is no + "official" image so we use the one from Bitnami. + +See `compose/dependencies.yaml` for the Docker Compose configuration +for each of these services. + +Other integration tests use additional servers such as Hadoop. +We will want to track down official images for those as well. + +## Guice and Lifecycle + +Nothing will consume more of your time than fighting with Druid's +Guice and Lifecycle mechanisms. These mechanisms are designed to do +exactly one thing: configure the Druid server. They are a nightmare +to use in other configurations such as unit or integration tests. + +### Guice Modules + +Druid has *many* Guice modules. There is no documentation to explain +which components are available from which modules, or their dependencies. +So, if one needs component X, one has to hunt through the source to +find the module that provides X. (Or, one has to "just know.") There +is no trick other than putting in the time to do the research, watching +things fail, and trying harder. + +In addition, modules have implicit dependencies: to use module Y you +must also include module Z. Again, there is no documentation, you have +to know or figure it out. + +The modules are designed to work only in the server, so they assume +the entire server is avaialble. Once we have a way that the modules +work in the server, we don't mess with it. But, in tests, we want +to use a subset because tests are clients, not a server. So, we end +up fighting to reuse a system that was designed for exactly one use +case: the server. The result is either a huge amount of time fiddling +to get things right or (as in the original integration tests), we just +include everything and pretend we are a server. + +There is no obvious solution, it is just a massive time sink at +present. + +### Druid Modules + +Many of the modules we want to use in integration test are +`DruidModule`s. These go beyond the usual Guice modules to provide +extra functionality, some of which is vital in tests: + +* The modules have depenencies injected from the "startup injector." +* The modules provide Jackson modules needed to serialized JSON. + +The `Initialization` class provides the mechanisms needed to work +with `DruidModule`s, but only when creating a server: that same class +has a strong opinion about which modules to include based on the +assumption that the process is a server (or a Druid tool which acts +like a server.) + +The code here refactored `Initialization` a bit to allow us to +use the functionality without being forced to accept all the default +server modules. The upside is that we don't end up having to fake the +tests to look like servers. The downside is the issue above: we have to +deal with the dependency nightmare. + +### Lifecycle Race Conditions + +Druid uses the `Lifecycle` class to manage object initialization. The +Lifecycle expects instances to be registered before the lifecycle +starts, after which it is impossible to register new instances. + +The lifecycle works out startup order based on Guice injection +dependencies. Thus, if a constructor is `X(Y y, Z y)` Guice knows +to create an `Y` and `Z` before creating `X`. `Lifecycle` leverages +this knowledge to start `Y` and `Z` before starting `X`. + +This works only if, during module creation, something has a +dependency on `X`. Else, if `X` is a `LazySingleton` it won't be +created until it is first used. But, by then, the `Lifecycle` will have +started and you'll get the dreaded "It doesn't work that way" error. + +### Guice and Lifecycle in Tests + +In the server, this works fine: there is exactly one usage of each +singleton, and the various modules have appearently been tweaked to +ensure every lifecycle-aware object is referenced (thus created, +this registerd in the lifecycle) by some other module. + +In tests, however, this system breaks down. Maven runs a series of +tests (via `failsafe`), each of which has any number of test methods. +The test driver is free to create any number of test class instances. + +When using the `Lifecycle` mechanism in tests, we would prefer to +set up the injector, and run the lifecycle, once per test class. This +is easy to do with the JUnit `@BeforeClass` annotation. But, when we +try this, the livecycle race condition issue slams us hard. + +Tests want to reference certain components, such as `DruidNodeDiscoveryProvider` +which require `CuratorFramework` which is provided by a module that +registers a component with the lifecycle. Because of the lazy singleton +pattern, `DruidNodeDiscoveryProvider` (and hence its dependenencies) +are created when first referenced, which occurs when JUnit instantiates +the test class, which happens after the Guice/Lifecycle setup in +`@BeforeClass`. And, we get our "It doesn't work that way" error. + +We can then try to move Guice/Lifecycle creation into the test class +constuctor, but then we'll watch as JUnit creates multiple instances +and we end up running initialization over and over. Further, it seems +there are race conditions when we do that (haven't figure out the +details), and we get strange errors. Further, we end up thrashing +the very complex initializaiton logic (which is a great stress test, +but we need to it only once, not on every test.) + +A hacky compromise is to add a caching layer: do the initialization in +the constructor, so we can inject the member variables, which creates +references, which causes the comonents to be created, which causes them +to register with the `Lifecycle` at the proper time. In the second +constructor call, we reuse the injector created in the first call. +Since we simply reuse the same singletons, we should not run into +Livecycle race conditions. The `@AfterClass` JUnit annotation is pressed +into service to shut down the lifecycle after all tests run. + +## Testing Tools And the Custom Node Role + +The Druid extension `druid-testing-tools` (Maven project +`extensions-core/testing-tools` provides an extension to be loaded +into the Druid image along with the Druid distribution and third-party +libraries. + +The `integration-tests` provides additional components (such as the +custom node role) that must be placed in the image, but uses an +entirely different mechanism. + +There is no documentation to explain why we do `testing-tools` one +way, the custom node role a different way. Is there a reason other than +the items were created by different people at different times who chose +to use different approaches? + +In an ideal world, `testing-tools` would contain the custom node role: +there would be a single way to provide test-only extensions. However, +since we must maintain backward compatibility with `integration-tests`, +and that module is a nightmare to modify, we must use a short-term +compromise. + +For now, we punt: we make a copy of `druid-testing-tools`, add the +`integraton-tools` custom node role, and call it `testing-tools-ex`. +See [`testing-tools/README`](../testing-tools/README.md) for the +details. + +## Integration Tests and `base-test` + +The `integration-tests` project contains the set of existing TestNG-based +tests as well as a large number of utilities used by the tests. +The revised framework adds its own utilities. + +The utilities speicfic to the new tests resides in the `base-test` +sub-project. We include the `integration-test` project to reusse its +utilities. + +This does create a potential conflict: as we convert tests, the tests +here will have the same name as tests in the `integration-test` +package, which causes duplicate class names on the class path: never +a good thing. + +The ideal solution would be to move the test utilities to a new +sub-project within `integration-tests` and have both the new and old test +projects include the resulting jar. + +For now, we use a "shadow" approach, we use the `org.apache.druid.testsEx` +package name for new tests so names do not conflict with the +`org.apache.druid.tests` name used in `integration-tests`. Eventually, +if all tests are renamed, we can rename the `testsEx` package back +to `tests`. + +In a few cases, the utilitiy classes make asumptions about the test +setup which does not match the new setup. In this case, we make a copy +of the class and apply needed changes. At present, only one class has this +issue: + +* `DruidClusterAdminClient` - interfaces with Docker using hard-coded + container names. + +The old versions are in `org.apache.druid.testing.utils` in +`integration-tests`, the new versions in `org.apache.druid.testing2.utils` +in this project. diff --git a/integration-tests-ex/docs/docker.md b/integration-tests-ex/docs/docker.md new file mode 100644 index 00000000000..afbc1064687 --- /dev/null +++ b/integration-tests-ex/docs/docker.md @@ -0,0 +1,302 @@ + + +# Docker Test Image for Druid + +Integration tests need a Druid cluster. While some tests support using +Kubernetes for the Quickstart cluster, most need a cluster with some +test-specific configuration. We use Docker Compose to create that cluster, +based on a test-oriented Docker image built by the `it-image` Maven module +(activated by the `test-image` profile.) +The image contains the Druid distribution, +unpacked, along with the MySQL and MariaDB client libaries and +and the Kafka protobuf dependency. Docker Compose is +used to pass configuration specific to each service. + +In addition to the Druid image, we use "official" images for dependencies such +as ZooKeeper, MySQL and Kafka. + +The image here is distinct from the +["retail" image](https://druid.apache.org/docs/latest/tutorials/docker.html) +used for getting started. The test image: + +* Uses a shared directory to hold logs and some configuration. +* Uses "official" images for dependencies. +* Assumes the wrapper Docker compose scripts. +* Has some additional test-specific extensions as defind in `it-tools`. + +## Build Process + +Assuming `DRUID_DEV` points to your Druid build directory, +to build the image (only): + +```bash +cd $DRUID_DEV/docker-tests/it-image +mvn -P test-image install +``` + +Building of the image occurs in four steps: + +* The Maven `pom.xml` file gathers versions and other information from the build. + It also uses the normal Maven dependency mechanism to download the MySQL, + MariaDB and + Kafka client libraries, then copies them to the `target/docker` directory. + It then invokes the `build-image.sh` script. +* `build-image.sh` adds the Druid build tarball from `distribution/target`, + copies the contents of `test-image/docker` to `target/docker` and + then invokes the `docker build` command. +* `docker build` uses `target/docker` as the context, and thus + uses the `Dockerfile` to build the image. The `Dockerfile` copies artifacts into + the image, then defers to the `test-setup.sh` script. +* The `test-setup.sh` script is copied into the image and run. This script does + the work of installing Druid. + +The resulting image is named `org.apache.druid/test:`. + +### Clean + +A normal `mvn clean` won't remove the Docker image because that is often not +what you want. Instead, do: + +```bash +mvn clean -P test-image +``` + +You can also remove the image using Docker or the Docker desktop. + +### `target/docker` + +Docker requires that all build resources be within the current directory. We don't want +to change the source directory: in Maven, only the target directories should contain +build artifacts. So, the `pom.xml` file builds up a `target/docker` directory. The +`pom.xml` file then invokes the `build-image.sh` script to complete the setup. The +resulting directory structure is: + +```text +/target/docker +|- Dockerfile (from docker/) +|- scripts (from docker/) +|- apache-druid--bin.tar.gz (from distribution, by build-image.sh) +|- MySQL client (done by pom.xml) +|- MariaDB client (done by pom.xml) +|- Kafka protobuf client (done by pom.xml) +``` + +Then, we invoke `docker build` to build our test image. The `Dockerfile` copies +files into the image. Actual setup is done by the `test-setup.sh` script copied +into the image. + +Many Dockerfiles issue Linux commands inline. In some cases, this can speed up +subsequent builds because Docker can reuse layers. However, such Dockerfiles are +tedious to debug. It is far easier to do the detailed setup in a script within +the image. With this approach, you can debug the script by loading it into +the image, but don't run it in the Dockerfile. Instead, launch the image with +a `bash` shell and run the script by hand to debug. Since our build process +is quick, we don't lose much by reusing layers. + +### Manual Image Rebuilds + +You can quick rebuild the image if you've previously run a Maven image build. +Assume `DRUID_DEV` points to your Druid development root. Start with a +Maven build: + +```bash +cd $DRUID_DEV/docker/test-image +mvn -P test-image install +``` + +Maven is rather slow to do its part. Let it grind away once to populate +`target/docker`. Then, as you debug the `Dockerfile`, or `test-setup.sh`, +you can build faster: + +```bash +cd $DRUID_DEV/docker/test-image +./rebuild.sh +``` + +This works because the Maven build creates a file `target/env.sh` that +contains the Maven-defined environment. `rebuild.sh` reads that +environment, then proceeds as would the Maven build. +Image build time shrinks from about a minute to just a few seconds. +`rebuild.sh` will fail if `target/env.sh` is missing, which reminds +you to do the full Maven build that first time. + +Remember to do a full Maven build if you change the actual Druid code. +You'll need Maven to rebuild the affected jar file and to recreate the +distribution image. You can do this the slow way by doing a full rebuild, +or, if you are comfortable with maven, you can selectively run just the +one module build followed by just the distribution build. + +## Image Contents + +The Druid test image adds the following to the base image: + +* A Debian base image with the target JDK installed. +* Druid in `/usr/local/druid` +* Script to run Druid: `/usr/local/launch.sh` +* Extra libraries (Kafka, MySQL, MariaDB) placed in the Druid `lib` directory. + +The specific "bill of materials" follows. `DRUID_HOME` is the location of +the Druid install and is set to `/usr/local/druid`. + +| Variable or Item | Source | Destination | +| -------- | ------ | ----- | +| Druid build | `distribution/target` | `$DRUID_HOME` | +| MySQL Connector | Maven repo | `$DRUID_HOME/lib` | +| Kafka Protobuf | Maven repo | `$DRUID_HOME/lib` | +| Druid launch script | `docker/launch.sh` | `/usr/local/launch.sh` | +| Env-var-to-config script | `docker/druid.sh` | `/usr/local/druid.sh` | + +Several environment variables are defined. `DRUID_HOME` is useful at +runtime. + +| Name | Description | +| ---- | ----------- | +| `DRUID_HOME` | Location of the Druid install | +| `DRUID_VERSION` | Druid version used to build the image | +| `JAVA_HOME` | Java location | +| `JAVA_VERSION` | Java version | +| `MYSQL_VERSION` | MySQL version (DB, connector) (not actually used) | +| `MYSQL_DRIVER_CLASSNAME` | Name of the MySQL driver (not actually used) | +| `CONFLUENT_VERSION` | Kafka Protobuf library version (not actually used) | + +## Shared Directory + +The image assumes a "shared" directory passes in additional configuration +information, and exports logs and other items for inspection. + +* Location in the container: `/shared` +* Location on the host: `/target/shared` + +This means that each test group has a distinct shared directory, +populated as needed for that test. + +Input items: + +| Item | Description | +| ---- | ----------- | +| `conf/` | `log4j.xml` config (optional) | +| `hadoop-xml/` | Hadoop configuration (optional) | +| `hadoop-dependencies/` | Hadoop dependencies (optional) | +| `lib/` | Extra Druid class path items (optional) | + +Output items: + +| Item | Description | +| ---- | ----------- | +| `logs/` | Log files from each service | +| `tasklogs/` | Indexer task logs | +| `kafka/` | Kafka persistence | +| `db/` | MySQL database | +| `druid/` | Druid persistence, etc. | + +Note on the `db` directory: the MySQL container creates this directory +when it starts. If you start, then restart the MySQL container, you *must* +remove the `db` directory before restart or MySQL will fail due to existing +files. + +### Third-Party Logs + +The three third-party containers are configured to log to the `/shared` +directory rather than to Docker: + +* Kafka: `/shared/logs/kafka.log` +* ZooKeeper: `/shared/logs/zookeeper.log` +* MySQL: `/shared/logs/mysql.log` + +## Entry Point + +The container launches the `launch.sh` script which: + +* Converts environment variables to config files. +* Assembles the Java command line arguments, including those + explained above, and the just-generated config files. +* Launches Java as "pid 1" so it will receive signals. + +### Run Configuration + +The "raw" Java environment variables are a bit overly broad and result +in copy/paste when a test wants to customize only part of the option, such +as JVM arguments. To assist, the image breaks configuration down into +smaller pieces, which it assembles prior to launch. + +| Enviornment Viable | Description | +| ------------------ | ----------- | +| `DRUID_SERVICE` | Name of the Druid service to run in the `server $DRUID_SERVICE` option | +| `DRUID_INSTANCE` | Suffix added to the `DRUID_SERVICE` to create the log file name. +Use when running more than one of the same service. | +| `DRUID_COMMON_JAVA_OPTS` | Java options common to all services | +| `DRUID_SERVICE_JAVA_OPTS` | Java options for this one service or instance | +| `DEBUG_OPTS` | Optional debugging Java options | +| `LOG4J_CONFIG` | Optional Log4J configuration used in `-Dlog4j.configurationFile=$LOG4J_CONFIG` | +| `DRUID_CLASSPATH` | Optional extra Druid class path | + +In addition, three other shared directories are added to the class path if they exist: + +* `/shared/hadoop-xml` - included itself +* `/shared/lib` - Included as `/shared/lib/*` to include extra jars +* `/shared/resources` - included itself to hold extra class-path resources + +### `init` Process + +Middle Manager launches Peon processes which must be reaped. +Add [the following option](https://docs.docker.com/compose/compose-file/compose-file-v2/#init) +to the Docker Compose configuration for this service: + +```text + init: true +``` + +## Extensions + +The following extensions are installed in the image: + +```text +druid-avro-extensions +druid-aws-rds-extensions +druid-azure-extensions +druid-basic-security +druid-bloom-filter +druid-datasketches +druid-ec2-extensions +druid-google-extensions +druid-hdfs-storage +druid-histogram +druid-kafka-extraction-namespace +druid-kafka-indexing-service +druid-kerberos +druid-kinesis-indexing-service +druid-kubernetes-extensions +druid-lookups-cached-global +druid-lookups-cached-single +druid-orc-extensions +druid-pac4j +druid-parquet-extensions +druid-protobuf-extensions +druid-ranger-security +druid-s3-extensions +druid-stats +it-tools +mysql-metadata-storage +postgresql-metadata-storage +simple-client-sslcontext +``` + +If more are needed, they should be added during the image build. + diff --git a/integration-tests-ex/docs/druid-config.md b/integration-tests-ex/docs/druid-config.md new file mode 100644 index 00000000000..5d715f43ca0 --- /dev/null +++ b/integration-tests-ex/docs/druid-config.md @@ -0,0 +1,138 @@ + + +# Druid Configuration + +In a normal install, Druid obtains configuration from properties files: + +* `/_common/common.runtime.properties` +* `//runtime.properties` + +In the container, Druid uses the same mechanism, though the common properties +file is empty. The container could simply mount the `runtime.properties` file. +However, doing so runs into the normal issues with Druid configuration: Druid +provides no form of inheritance: we'd have to copy/paste the same properties +over and over, which would be a maintenance headache. + +Instead, the images use the same technique as the +[production Docker image](https://druid.apache.org/docs/latest/tutorials/docker.html): +we pass in a large number of environment variables. + +The test configuration extends the production set to include extra +variables. Thus there are two kinds: + +* General configuration (capitalized) +* Druid configuration file settings (lower case) + +## Configuration Flow + +We use `docker-compose` to gather up the variables. From most specific +(highest priority) to most general, configuration comes from: + +* An environment variable set by the script which launches Docker Compose. + (Use sparingly, only for different test "modes" such as choosing the + DB driver, when we will use a different mode across diffrerent test runs.) +* As in-line settings in the `environment` section in the Docker Compose + definition for each service. +* In the service-specific `compose/environment-configs/.env` file. +* In the common `compose/environment-configs/common.env` file. + +Make test-specific changes in the test-specific Docker compose file. Make +changes to the `*.env` files only if you are certain that the change should +apply to all tests. An example is when we change something in our product +configs. + +The set of defined environment variables starts with the +`druid/conf/single-server/micro-quickstart` settings. It would be great to generate +these files directly from the latest quickstart files. For now, it is a manual +process to keep the definitions in sync. + +These are defined in a hierarchy: + +* `common.env` - roughly equivalent to the `_common` configuration area in Druid: + contains definitions common to all Druid services. Services can override any + of the definitions. +* `.env` - base definitions for each service, assume it runs stand-alone. + Adjust if test cluster runs multiple instances. Rougly equivalent to the + service-specific `runtime.properties` file. +* `docker-compose.yaml` - test-specific settings. + +The `launch.sh` script converts these variables to config files in +`/tmp/conf/druid`. Those files are then added to the class path. + +## Druid Config Settings + +To set a Druid config variable, replace dots in the name with underscores. + +In the usual properties file: + +```text +druid.auth.basic.common.maxSyncRetries=20 +``` + +In an environment config file: + +```text +druid_auth_basic_common_maxSyncRetries=20 +``` + +```text + environment: + - druid_auth_basic_common_maxSyncRetries=20 +``` + +For everyone's sanity, please include a comment to explain the reason for +the setting if it differs from the Quickstart defaults. + +## Special Config Variables + +The test configuration goes beyond the production Druid image configuration +to add several extensions specfically for tests. These are variables which +handle some specific part of the configuration to avoid what would otherwise +require redundant copy/paste. See the [Docker section](docker.md) for the +details. + +## Shared Directory + +Druid configuration includes not just the config files, but also items +on the Druid class path. These are provided via a `shared` directory mounted +into the container at `/shared`. +The shared directory is built in the `target/` folder for each test +category. + +The `launch.sh` script fills in a number of implicit configuration items: + +| Item | Description | +| ---- | ----------- | +| Heap dump path | Set to `${SHARED}/logs/` | +| Log4J config | Optional at `${SHARED}/conf/log4j.xml` | +| Hadoop config | Optional at `${SHARED}/hadoop-xml` | +| Extra libraries | Optional at `${SHARED}/lib` | +| Extra resources | Optional at `${SHARED}/resources` | + +`${SHARED}/resources` is the place to put things like a custom `log4j2.xml` +file. + +## Security Setup + +Tests can run with or without security enabled. (Security setup is a work in progress, +the prior integration tests enabled security for all tests.) + +* `auth.env` - Additional definitions to create a secure cluster. Also requires that + the client certificates be created. Add this to tests which test security. diff --git a/integration-tests-ex/docs/guide.md b/integration-tests-ex/docs/guide.md new file mode 100644 index 00000000000..795a725e690 --- /dev/null +++ b/integration-tests-ex/docs/guide.md @@ -0,0 +1,241 @@ + + +# Test Creation Guide + +You've played with the existing tests and you are ready to create a new test. +This section walks you through the process. If you are converting an existing +test, then see the [conversion guide](conversion.md) instead. The details +of each step are covered in other files, we'll link them from here. + +## Category + +The first quesetion is: should your new test go into an existing category, +or should you create a new one? + +You should use an existing category if: + +* Your test is a new case within an obviously-existing category. +* Your test needs the same setup as an existing category, and is quick + to run. Using the existing category avoids the need to fire up a + Docker cluster just for your test. + +You should create a new category if: + +* Your test uses a customized setup: set of services, service + configuration, set of external dependencies, instead. +* Your test will run for an extended time, and is best run in + parallel with other tests in a build envrionment. Your test + can share a cluster configuration with an existing test, but + the new category allows the test to run by itself. + +When your test *can* reuse an existing cluser definition, then the question is +about time. It takes significan time (minutes) to start a Docker cluster. We clearly +don't want to pay that cost for a test that runs for seconds, if we could just add the +test to another category. On the other hand, if you've gone crazy and added a huge +suite of tests that take 20 minutes to run, then there is a huge win to be had by +running the tests in parallel, even if they reuse an existing cluster configuration. +Use your best judgment. + +The existing categories are listed in the +`org.apache.druid.testsEx.categories` package. The classes there represent +[JUnit categories]( +https://junit.org/junit4/javadoc/4.12/org/junit/experimental/categories/Categories.html). +See [Test Category](tests.md#Test+Category) for details. + +If you create a new category, but want to reuse the configuration of +an existing category, add the `@Cluster` annotation as described in the above +link. Note: be sure to link to a "base" category, not to a category that, itself, +has a `@Cluster` annotation. + +If you use the `@Cluster` annotation, you must also add a mapping in the +`cluster.sh` file. See the top of the file for an example. + +## Cluster Configuration + +If you create a new category, you must define a new cluster. There are two parts: + +* Docker compose +* Test configuration + +### Docker Compose + +Create a new folder: `custer/`, then create a `docker-compose.yaml` file +in that folder. Define your cluster by borrowing heavily from existing files. +See [compose](compose.md) for details. + +The only trick is if you want to include a new external dependency. The preferred +approach is to use an "official" image. If you must, you can create a custom image +in the `it-image` module. (We've not yet done that, so if you need a custom image, +let us know and we'll figure it out.) + +### Test Configuration + +Tests need a variety of configuration information. This is, at present, more +complex than we might like. You will at least need: + +* Describe the Docker Compose cluster +* Provide test-specific properties + +You may also need: + +* Test-specific Guice modules +* Environment variable bindings to various properties +* MySQL statements to pre-populate the Druid metastore DB +* And so on. + +### Test Config File + +The cluster and properties are defined in a config file. Create a folder +`src/test/resources/cluster/`. Then add a file called `docker.yaml`. +Crib the contents from the same category from which you borrowed the Docker +Compose definitions. Strip out properties and metastore statements you don't +need. Add those you do need. See [Test Configuration](test-config.md) for the +gory details of this file. + +### Test Config Code + +You may also want to customize Guice, environment variable bindings, etc. +This is done in the [test setup](tests.md#Initialization) method in your test. + +## Start Simple + +There are *many* things that can go wrong. It is best to start simple. + +### Verify the Cluster + +Start by ensuring your cluster works. + +* Define your cluster as described above. Or, pick one to reuse. +* Verify the cluster using `it.sh up `. +* Look at the Docker desktop UI to ensure the cluster says up. if not, + track down what went wrong. Look at both the Docker (stdout) and + Druid (`target//logs/.log`) files. + +### Starter Test + +Next, create your test file as described above and in [Tests](tests.md). + +* Create the test class. +* Add the required annotations. +* Create a simple test function that just prints "hello, world". +* Create your `docker.yaml` file as decribed above. +* Start your cluster, as described above, if not already started. +* Run the test from your IDE. +* Verify that the test "passes" (that is, it prints the message.) + +If so, then this means that your test connected to your custer and +verified the health of all the services declared in your `docker.yaml` file. + +If something goes wrong, you'll know it is in the basics. Check your +cluster status. Double-check the `docker.yaml` structure. Check ports. +Etc. + +### Client + +Every test is a Druid client. Determine which service API you need. Find an +existing test client. The `DruidClusterAdminClient` is the "modern" way to +interact with the cluster, but thus far has a limited set of methods. There +are older clients as well, but they tend to be quirky. Feel free to extend +`DruidClusterAdminClient`, or use the older one: whatever works. + +Inject the client into your test. See existing tests for how this is done. + +Revise your "starter" test to do some trivial operation using the client. +Retest to ensure things work. + +### Test Cases + +From here, you can start writing tests. Explore the existing mechanisms +(including those in the original `druid-integration-tests` module which may +not yet have been ported to the new framework yet.) For example, there are +ways to store specs as files and parameterize them in tests. There is a +syntax for running queries and specifying expected results. + +You may have to create a new tool to help with your test. If you do, +try to use the new mechanisms, such as `ResolvedClusterConfig` rather than +using the old, cumbersome ones. Post questions in Slack so we can help. + +### Extensions + +Your test may need a "non-default" extension. See [Special Environment Variables]( +compose.md#Special+Environment+Variables) for how to specify test-specific +extensions. (Hint: don't copy/paste the full load list!) + +Extensions have two aspects in ITs. They act like extensions in the Druid servers +running in Docker. So, the extension must be avaialble in the Docker image. All +standard Druid extensions which are available in the Druid distribution, are also +available in the image. The may not be enabled, however. Hence the need to define +the custom load list. + +Your test may use code from the extension. To the *tests*, however, the extension +is just another jar: it must be listed in the `pom.xml` file. There is no such +thing as a "Druid extensions" to the tests themselves. + +If you test an extension that is *not* part of the Druid distributeion, then it +has to get into the image. Reach out on the slack mailing list so we can discuss +solutions (such as mounting a directory that contains the extension). + +### Retries + +The old IT framework was very liberal in its use of retries. Retires were +used to handle: + +* the time lag in starting a cluster, +* the latency inherent in events propagaing through a distributed system + (such as when segments get published), +* random network failures, +* flaky tests. + +The new framework takes a stricter view. The framework itself will ensure +service are ready (using the Druid API for that purpose.) If a server reports +itself ready, but still fails on one of your API calls, then we've got a bug +to fix. Don't use retries to work around this issue because users won't know +to do this. + +In the new framwork, tests should not be flaky. Flaky tests are a drag on +development; they waste time. If your test is flaky, please fix it. Don't count +on the amount of times things take: a busy build system will run much slower than +your dedicated laptop. And so on. + +Ideally, Druid would provide a way to positively confirm that an action has +occurred. Perhaps this might be a test-only API. Otherwise, a retry is fine, but +should be coded into your test. (Or, better, implemented in a client.) Do this only +if we document that, for that API, users should poll. Otherwise, again, users of +the API under test won't know to retry, and so the test shouldn't do so either. + +This leaves random failures. The right place to handle those is in the client, +since they are independent of the usage of the API. + +The result of the above is that you should not need (or use) the `ITRetryUtil` +mechanism. No reason for your test to retry 240 times if something is really wrong +or your test is flaky. + +This is an area under development. If you see a reason to retry, lets discuss it +and put it in the proper place. + +### Travis + +Run your tests in the IDE. Try them using `it.sh test `. If that passes +add the test to Travis. The details on how to do so are still being worked out. +Likely, you will just copy/paste an existing test "stanza" to define your new +test. Your test will run in parallel with all other IT categories, which is why +we offered the advice above: the test has to have a good reason to fire up yet +another build task. + diff --git a/integration-tests-ex/docs/history.md b/integration-tests-ex/docs/history.md new file mode 100644 index 00000000000..1a35be30323 --- /dev/null +++ b/integration-tests-ex/docs/history.md @@ -0,0 +1,96 @@ + + + +## History + +This revision of the integration test Docker scripts is based on a prior +integration test version, which is, in turn, based on +the build used for the public Docker image used in the Druid tutorial. If you are familiar +with the prior structure, here are some of the notable changes. + +* Use of "official" images for third-party dependencies, rather than adding them + to the Druid image. (Results in *far* faster image builds.) +* This project splits the prior `druid-integration-tests` project into several parts. This + project holds the Druid Docker image, while sibling projects hold the cluster definition + and test for each test group. + This allows the projects to better utilize the standard Maven build phases, and allows + better partial build support. +* The prior approach built the Docker image in the `pre-integration-test` phase. Here, since + the project is separate, we can use the Maven `install` phase. +* The prior structure ran *before* the Druid `distribution` module, hence the Druid artifacts + were not available, and the scripts did its own build, which could end up polluting the + Maven build cache. This version runs after `distribution` so it can reuse the actual build + artifacts. +* The `pom.xml` file in this project does some of the work that that `build_run_cluster.sh` + previously did, such as passing Maven versions into Docker. +* The work from the prior Dockerfile and `base-setup.sh` are combined into the revised + `base-setup.sh` here so that the work is done in the target container. +* Since the prior approach was "all-in-one", it would pass test configuration options into + the container build process so that the container is specific to the test options. This + project attempts to create a generic container and instead handle test-specific options + at container run time. +* The detailed launch commands formerly in the Dockerfile now reside in + `$DRUID_HOME/launch.sh`. +* The prior version used a much-extended version of the public launch script. Those + extensions moved into `launch.sh` with the eventual goal of using the same launch + scripts in both cases. +* The various `generate_*_cert.sh` scripts wrote into the source directory. The revised + scripts write into `target/shared/tls`. +* The shared directory previously was in `~/shared`, but that places the directory outside + of the Maven build tree. The new location is `$DRUID_DEV/docker/base-docker/target/shared`. + As a result, the directory is removed and rebuild on each Maven build. The old location was + removed via scripts, but the new one is very clearly a Maven artifact, and thus to be + removed on a Maven `clean` operation. +* The prior approach had security enabled for all tests, which makes debugging hard. + This version makes security optional, it should be enabled for just a security test. +* The orginal design was based on TestNG. Revised tests are based on JUnit. +* The original tests had "test groups" within the single directory. This version splits + the former groups into projects, so each can have its own tailored cluster definition. +* Prior images would set up MySQL inline in the container by starting the MySQL engine. + This led to some redundancy (all images would do the same thing) and also some lost + work (since the DBs in each container are not those used when running.) Here, MySQL + is in its own image. Clients can update MySQL as needed using JDBC. +* Prior code used Supervisor to launch tasks. This version uses Docker directly and + runs one process per container (except for Middle Manager, which runs Peons.) + +## History + +The current work builds on the prior integration tests, with changes to +simplify and speed up the process. + +* The prior tests required a separate Docker build for each test "group" + Here, the former groups are sub-projects. All use the same Docker image. +* The prior code used the long-obsolte TestNG. Tests here use JUnit. +* The prior test used a TestNG suite to create test intances and inject + various items using Guice. This version uses an `Initializer` class to + do roughly the same job. +* The prior tests required test configuration be passed in on the command + line, which is tedious when debugging. This version uses a cluster + configuation file instead. +* The prior version perfomed MySQL initialization in the Docker container. + But, since each test would launch multiple containers, that work was + done multiple times. Here the work is done by the test itself. +* The prior version had a single "shared" directory for all tests in + `~/shared`. This version creates a separate shared folder for each + test module, in `/target/shared`. This ensures that Maven will + delete everything between test runs. +* This version removes many of the `druid-` prefixes on the container + names. We assume that the cluster runs as the only Docker app locally, + so the extra naming just clutters things. \ No newline at end of file diff --git a/integration-tests-ex/docs/maven.md b/integration-tests-ex/docs/maven.md new file mode 100644 index 00000000000..6d17aebe452 --- /dev/null +++ b/integration-tests-ex/docs/maven.md @@ -0,0 +1,255 @@ + + +# Maven Structure + +The integration tests are built and run as part of Druid's Maven script. +Maven itself is used by hand, and as part of the [Travis](travis.md) build +proces. Running integration tests in maven is a multi-part process. + +* Build the product `distribution`. +* Build the test image. The tests run against the Maven-created Druid build, + and so appear in the root `pom.xml` file *after* the `distribution` + project which builds the Druid tarball. +* Run one or more ITs. Each Maven run includes a single test category and its + required Druid cluster. + +Travis orchestrates the above process to run the ITs in parallel. When you +run tests locally, you do the above steps one by one. You can, of course, reuse +the same disribution for multiple image builds, and the same image for multiple +test runs. + +## Build the Distribution and Image + +Use the following command to run the ITs, assuming `DRUID_DEV` points +to your Druid development directory: + +```bash +cd $DRUID_DEV +mvn clean package -P dist,test-image,skip-static-checks \ + -Dmaven.javadoc.skip=true -DskipUTs=true +``` + +The various pieces are: + +* `clean`: Remove any existing artifacts, and any existing Docker image. +* `install`: Build the Druid code and write it to the local Maven repo. +* `-P dist`: Create the Druid distribution tarball by pulling jars from + the local Maven repo. +* `-P test-image`: Build the Docker images by grabbing the Druid tarball + and pulling additional dependencies into the local repo, then stage them + for Docker. +* Everything else: ignore parts of the build not needed for the ITs, such + as static checks, unit tests, Javadoc, etc. + +Once you've done the above once, you can do just the specific part you want +to repeat during development. See below for details. + +See [quickstart](quickstart.md) for how to run the two steps separately. + +## Run Each Integration Test Category + +Each pass through Maven runs a single test category. Running a test category +has three parts, spelled out in Maven: + +* Launch the required cluster. +* Run the test category. +* Shut down the cluster. + +Again, see [quickstart](quickstart.md) for how to run the three steps separately, +and how to run the tests in an IDE. + +To do the task via Maven: + +```bash +cd $DRUID_DEV +mvn verify -P docker-tests,skip-static-checks,IT- \ + -Dmaven.javadoc.skip=true -DskipUTs=true +``` + +The various pieces are: + +* `verify`: Run the steps up to the one that checks the output of the ITs. Because of + the extra cluster step in an IT, the build does not fail if an IT failse. Instead, + it continues on to clean up the cluster, and only after that does it check test + sucess in the `verify` step. +* `` selects the category to run. + +The profiles allow you to build the test image once during debugging, +and reuse it across multiple test runs. (See [Debugging](debugging.md).) + +## Dependencies + +The Docker image inclues three third-party dependencies not included in the +Druid build: + +* MySQL connector +* MariaDB connector +* Kafka Protobuf provider + +We use dependency rules in the `test-image/pom.xml` file to cause Maven to download +these dependencies into the Maven cache, then we use the +`maven-dependency-plugin` to copy those dependencies into a Docker directory, +and we use Docker to copy the files into the image. This approach avoids the need +to pull the dependency from a remote repository into the image directly, and thus +both speeds up the build, and is kinder to the upstream repositories. + +If you add additional dependencies, please follow the above process. See the +`pom.xml` files for examples. + +## Environment Variables + +The build environment users environment variables to pass information to Maven. +Maven communicates with Docker and Docker Compose via environment variables +set in the `exec-maven-plugin` of various `pom.xml` files. The environment +variables then flow into either the Docker build script (`Dockerfile`) or the +various Docker Compose scripts (`docker-compose.yaml`). It can be tedious to follow +this flow. A quick outline: + +* The build environment (such as Travis) sets environment variables, or passes values + to maven via the `-d= + +# Future Work + +The present version establishes the new IT framework. Work completed to +date includes: + +* Restructure the Docker images to use the Druid produced from the + Maven build. Use "official" images for dependencies. +* Restructure the Docker compose files. +* Create the cluster configuration mechanisms. +* Convert one "test group" to a sub-module: "high-availability". +* Create the `pom.xml`, scripts and other knick-knacks needed to tie + everything together. +* Create the initial test without using security settings to aid + debugging. + +However, *much* work remains: + +* Convert remaining tests. +* Decide when we need full security. Convert the many certificate + setup scripts. +* Support cluster types other than Docker. + +## Open Tasks + +The following detail items are open: + +* Disable a test if the `disabled` type is set in the test configuration + file. Apply it to disable the HA tests for all but Docker. +* Handle missing config files: generate a "dummy" that is disabled. +* When launching a container build or test run from Maven, write + environment variables to a `target/env.sh` file so that the user + doesn't have to find them manually to run the helper scripts. +* There is some redundancy in each test group project. Figure out + solutions: + * The `cluster.sh` script + * Boilerplate in the `pom.xml` file. +* Move test data from `/resources` to `/shared/data`. Build up the + data directory from multiple sources during cluster launch. +* Sort out which data and spec files are actually used. Remove those + which are not used. Sort the files by test-specific and shared + across tests by moving them into different directories. + +## Later Tasks + +The "public" and "integration test" versions of the Docker images have diverged significantly, +which makes it harder to "test what we ship." Differences include: + +* Different base image +* Different ways to set up dependencies. +* Different paths within the container. +* Different launch scripts. +* The test images place Druid in `/usr/local`, the public images in `/opt`. + +The tests do want to do things beyond what the "public" image does. However, this should +not require a fork of the builds. To address this issue: + +* Extend this project to create a base common to the "public" and integration test images. +* Extend the integration test image to build on top of the public image. diff --git a/integration-tests-ex/docs/quickstart.md b/integration-tests-ex/docs/quickstart.md new file mode 100644 index 00000000000..1a3fcb22f7b --- /dev/null +++ b/integration-tests-ex/docs/quickstart.md @@ -0,0 +1,201 @@ + + +# Quickstart + +If you just need to know how to build, run and use the tests, this +is the place. You can refer to the detailed material later as you +add new tests or work to improve the tests. + +## Example Test + +When first learning the framework, you can try thing out using the +`HighAvailability` test. Of the tests converted thus far, it is the +one that runs for the shortest time (on the order of a minute or two.) + +## Working with Individual Tests + +To work with tests for development and debugging, you can break the +above all-in-one step into a number of sub-steps. + +* [Build Druid](#Build Druid). +* [Build the Docker image](#Build the Docker Image). +* [Start a cluster](#Start a Cluster). +* [Run a test from the command line](#Run a Test from the Command Line). +* [Run a test from an IDE](#Run a Test from an IDE). +* [Stop the cluster](#Stop the Cluster). +* [Clean up](#Clean Up). + +## Build Druid + +The integration tests start with a Druid distribution in `distribution/target`, +which you can build using your preferred Maven command line. Simplest: + +``` +cd $DRUID_DEV +./it.sh dist +``` + +Or, in detail: + +For example: +To make the text a bit simpler, define a variable for the standard settings: + +```bash +export MAVEN_IGNORE=-P skip-static-checks,skip-tests -Dmaven.javadoc.skip=true +``` + +```bash +mvn clean package -P dist $MAVEN_IGNORE -T1.0C +``` + +Hint: put this into a script somewhere, such as a `~/bin` directory and +add that to your `PATH`. A good name is `bdru` (Build DRUid). +Here is the full script: + +```bash +#! /bin/bash + +mvn clean package -P dist,skip-static-checks,skip-tests \ + -Dmaven.javadoc.skip=true -T1.0C $* +``` + +Now you can run the any Druid build with the above script. To resume a +build: + +```bash +> bdru -rf :foo +``` + +## Build the Docker Image + +You must rebuild the Docker image whenever you rebuild the Druid distribution, +since the image includes the distribution. You also will want to rebuild the +image if you change the `it-image` project which contains the build scripts. + +```bash +./it.sh image +``` + +In detail, and assuming `DRUID_DEV` points to your Druid build directory, +to build the image (only): + +```bash +cd $DRUID_DEV/integration-tests-ex/image +mvn install -P test-image $MAVEN_IGNORE +``` + +The above has you `cd` into the project to avoid the need to disable all the +unwanted bits of the Maven build. + +See [this page](docker.md) for more information. + +## Run an IT from the Command Line + +```bash +./it.sh test +``` + +Or, in detail: + +```bash +mvn verify -P docker-tests,IT- -pl :druid-it-cases \ + -P skip-static-checks,skip-tests -Dmaven.javadoc.skip=true -DskipUTs=true +``` + +Where `` is one of the test categories. + +Or + +```bash +cd $DRUID_DEV/integration-tests-ex/cases +mvn verify -P skip-static-checks,docker-tests,IT- \ + -Dmaven.javadoc.skip=true -DskipUTs=true \ + -pl :druid-it-cases +``` + +If the test fails, find the Druid logs in `target/shared/logs` within the +test group project. + +## Start a Cluster + +The previous generation of tests were organized into TestNG groups. This +revision moves those groups into Maven modules. Each group has a distinct +cluster configuration. (In fact, it is the cluster configuration which defines +the group: we combine all tests with the same configuration into the same module.) +So, to start a cluster, you have to pick a group to run. See +[this list](maven.md#Modules) for the list of groups. + +```bash +./it.sh up +``` + +Or, in detail: + +```bash +cd $DRUID_DEV/integration-tests-ex/cases +./cluster.sh up +``` + +You can use Docker Desktop to monitor the cluster. Give things about 30 seconds +or a minute: if something is going to fail, it will happen during starup and you'll +see that one or more containers exited unexpectedly. + +Remember to first shut down any Druid cluster you may already be running on +your machine. + +See [this page](docker.md) for more information. + +## Run a Test from an IDE + +To run an IT in your IDE: + +* Find the IT to run. +* Run it as a JUnit test. + +The tests are specifically designed to require no command-line setup: you can +just run them directly. + +## Stop the Cluster + +Once you are done with your cluster, you can stop it as follows: + + +```bash +./it.sh down +``` + +Or, in detail: + +```bash +cd $DRUID_DEV/integration-tests-ex/ +./cluster.sh down +``` + +## Clean Up + +You can remove the Docker image when you no longer need it: + +```bash +cd $DRUID_DEV +mvn clean -P test-image +``` + +It is usually fine to skip this step: the next image build will replace +the current one anyway. diff --git a/integration-tests-ex/docs/runtime.md b/integration-tests-ex/docs/runtime.md new file mode 100644 index 00000000000..c276eee207e --- /dev/null +++ b/integration-tests-ex/docs/runtime.md @@ -0,0 +1,105 @@ + + +## Test Runtime Behavior + +This section explains how the various configuration pieces come together +to run tests. + +See also: + +* [Docker setup](docker.md) +* [Docker Compose setup](compose.md) +* [Test setup](test-config.md) + +## Shared Resources + +This module has a number of folders that are used by all tests: + +* `compose`: A collection of Docker Compose scripts that define the basics + of the cluster. Each test "inherits" the bits that it needs. +* `compose/environment-configs`: Files which define, as environment variables, + the runtime properties for each service in the cluster. (See below + for details.) +* `assets`: The `log4j2.xml` file used by images for logging. + +## Container Runtime Structure + +The container itself is a bit of a hybrid. The Druid distribution, along +with some test-specific extensions, is reused. The container also contains +libraries for Kafka, MySQL and MariaDB. + +Druid configuration is passed into the container as environment variables, +and then converted to a `runtime.properties` file by the container launch +script. Though a bit of a [Rube Goldberg](https://en.wikipedia.org/wiki/Rube_Goldberg) +mechanism, it does have one important advantage over the usual Druid configs: +we can support inheritance and overrides. The various `.env` files +provide the standard configurations. Test-specific Docker Compose files can +modify any setting. + +The container mounts a shared volume, defined in the `target/shared` directory +of each test module. This volume can provide extra libraries and class path +items. The one made available by default is `log4j2.xml`, but tests can add +more as needed. + +Container "output" also goes into the shared folder: logs, "cold storage" +and so on. + +Each container exposes the Java debugger on port 8000, mapped to a different +host port for each container. + +Each container exposes the usual Druid ports so you can work with the +container as you would a local cluster. Two handy tools are the Druid +console and the scriptable [Python client](https://github.com/paul-rogers/druid-client). + +## Test Execution + +Tests run using the Maven [failsafe](https://maven.apache.org/surefire/maven-failsafe-plugin/) +plugin which is designed for integration tests. The Maven phases are: + +* `pre-integration-test`: Starts the test cluster with `cluster.sh up` using Docker Compose. +* `integration-test`: Runs tests that start or end with `IT`. +* `post-integration-test`: Stops the test cluster using `cluster.sh down` +* `verify`: Checks the result of the integration tests. + +See [this example](https://maven.apache.org/surefire/maven-failsafe-plugin/examples/junit.html) +for JUnit setup with failsafe. + +The basic process for running a test group (sub-module) is: + +* Cluser startup builds a `target/shared` directory with items to be mounted + into the containers, such as the `log4j2.xml` file, sample data, etc. + The shared directory also holds log files, Druid persistent storage, + the metastore (MySQL) DB, etc. See `test-image/README.md` for details. +* The test is configured via a `druid-cluster/compose.yaml` file. + This file defines the services to run and their configuration. +* The `cluster.sh up` script builds the shared directory, loads the env vars + defined when the image was created and starts the cluster. +* Tests run on the local host within JUnit. +* The `Initialization` class loads the cluster configuration (see below), + optionally populates the Druid metadata storage, and is used to + inject instances into the test. +* The individual tests run. +* The `cluster.sh down` script shuts down the cluster. + +`cluster.sh` uses the generated `test-image/target/env.sh` for versions and +and other environment variables. This ensures that tests run with the same +versions used to build the image. It also simplifies the Maven boilerplate to +be copy/pasted into each test sub-project. + diff --git a/integration-tests-ex/docs/scripts.md b/integration-tests-ex/docs/scripts.md new file mode 100644 index 00000000000..2c9ceb2a874 --- /dev/null +++ b/integration-tests-ex/docs/scripts.md @@ -0,0 +1,74 @@ + + +# Scripts + +The IT framework uses a number of scripts and it can be a bit of a mystery +what each one does. This guide lists each of them. + +## `integration-tests-ex` + +* `it.sh` - Utility to perform many IT-related actions such as building Druid, + running ITs, starting a cluster, etc. Use `it.sh help` to see the list of commands. + +### `it-image` + +* `build-image.sh` - Internal script to (you guessed it), build the image. + Creates the `target/env.sh` file above, then invokes `docker-build.sh`. +* `rebuild.sh` - Rebuilds the image after the above script has created the + `env.sh` file. Used to debug changes to the image build itself: use `rebuild.sh` + rather than waiting for Maven to do its thing. +* `docker-build.sh` - Internal script to set up the needed environment + variables and invoke Docker to build the image. Modify this if you ned to change + the information passed into Docker via the command line. +* `docker/Dockerfile` - Docker script to build the image. +* `docker/test-setup.sh` - Script copied into the image at build time and run to + set up the image. Keeps the `Dockerfile` simpler (and, it is somewhat easier to + debug this script than the `Dockerfile` itself.) +* `docker/launch.sh` - Container entrypoint which runs inside the container. + Sets up configuration and calls `druid.sh` to launch Druid itself. +* `druid.sh` - Creates a Druid configuration file from environment variables, + then runs Druid within the container. +* `target/env.sh` - Created when the image is built. Provides environment + variables for things like the image name, versions and so on. Used to + quickly rebuild the image (see [Maven configuration](docs/maven.md)) and + to launch tests. + +### `test-cases` + +* `cluster//*.yaml` - Base Docker Compose scripts that define the "standard" + Druid cluster. Tests use these files to avoid redundant copy/past of the + standard items. +* `cluster.sh` - Launches or tears down a cluster for a test. Called from Maven + and `it.sh`. Can be used manually. See below. + +The options for `cluster.sh` are: + +```bash +cluster.sh [-h|help|up|down|status|compose-cmd] category +``` + +* `up` - starts the cluster. +* `down` - shuts down the cluster. +* `status` - displays cluster status for debugging. Expecially useful for debugging + issues in Travis where we cannot directly inspect the Docker cluster itself. +* `help` - repeats the usage line. +* Others - passes the command on to Docker Compose. +* `category` - the test category to launch. Performs mapping from the category name + to cluster name when categories share definitions. diff --git a/integration-tests-ex/docs/test-config.md b/integration-tests-ex/docs/test-config.md new file mode 100644 index 00000000000..4a5c7a583f7 --- /dev/null +++ b/integration-tests-ex/docs/test-config.md @@ -0,0 +1,588 @@ + + +## Test Configuration + +Tests typically need to understand how the cluster is structured. +To create a test, you must supply at least three key components: + +* A `cluster//docker-compose.yaml` file that launches the desired cluster. + (The folder name `` becomes the application name in Docker.) +* A `src/test/resources/cluster//docker.yaml` file that describes the cluster + for tests. This file can also include Metastore SQL statements needed to + populate the metastore. +* The test itself, as a JUnit test that uses the `Initializer` class to + configure the tests to match the cluster. + +This section explains the test configuration file which defines the test +cluster. + +Note that you can create multiple versions of the `docker.yaml` file. For example, +you might want to create one that lists hosts and credentials unique to your +debugging environment. You then use your custom version in place of the standard +one. + +## Cluster Types + +The integration tests can run in a variety of cluster types, depending +on the details of the test: + +* Docker Compose: the normal configuration that all tests support. +* Micro Quickstart: allows for a manual cluster setup, if, say, you + want to run services in your IDE. Supported by a subset of tests. +* Kubernetes: (Details needed.) + +Each cluster type has its own quirks. The job of the tests's cluster configuration +file is to communicate those quirks to the test. + +Docker and Kubernetes use proxies to communicate to the cluster. Thus, the host +known to the tests is different than the hosts known within the cluster. Ports +may also are mapped differently "outside" than "inside." + +Clusters outside of Docker don't provide a good way to start and stop +services, so tests that want to do that (to, say, test high availability) +can't run except in a Docker cluster. + +### Specify the Cluster Type + +To reflect this, tests provide named configuration files. The configuration +itself is passed in via the environment: + +```bash +export TEST_CONFIG=quickstart +``` + +```bash +java ... -DtestConfig=quickstart +``` + +The system property taskes precedence over the environment variable. +If neither are set, `docker` is the default. The configuration file +itself is assumed to be a resource named `/yaml/.yaml`. + +As a debug aide, a test can specify and ad-hoc file in the file system +to load for one-off special cases. See `Initialization.Builder` for +details. + +## Cluster Configuration Files + +Cluster configuration is specified in a file for ease of debugging. Since +configuration is in a file (resource), and not in environment variables or +system properties, you +should need no special launch setup in your IDE to run a test that uses +the standard Docker Compose cluster for that test. + +The configuration file has the same name as the cluster type and resides on +the class path at `/yaml/.yaml` and in the source tree at +`/src/test/resources/yaml/.yaml`. The standard names are: + +* `docker.yaml`: the default and required for all tests. Describes a Docker + Compose based test. +* `k8s.yaml`: a test cluster running in Kubernetes. (Details needed.) +* `local.yaml`: a local cluser such as Micro Quickstart cluster. (Details needed.) +* `.yaml`: custom cluster configuration. + +Configuration files support include files. Most of the boiler-plate configuration +should appear in commmon files. As a result, you should only need to specify +test-specific differences in your `docker.yaml` file, with all else obtained +from the included files. + +## Configuration File Syntax + +The configuration is a [YAML](https://yaml.org/spec/1.2.2/) file that +has a few top-level properties and an entry for each service in your +cluster. + +### `type` + +```yaml +type: docker|k8s|local|disabled +``` + +The type explains the infrastructure that runs the cluster: + +* `docker`: a cluster launched in Docker, typically via Docker Compose. + A proxy host is needed. (See below.) +* `k8s`: a cluster run in Kubernets. (Details needed). A proxy host + is needed. +* `local`: a cluster running as processes on a network directly reachable + by the tests. Example: a micro-quickstart cluster running locally. +* `disabled`: the configuration is not supported by the test. + +The `disabled` type is handy for tests that require Docker: you can say that +the test is not available when the cluster is local. + +If the test tries to load a cluster name that does not exist, a "dummy" +configuration is loaded instead with the type set to `disabled`. + +The type is separate from the cluster name (as explained earlier): there +may be multiple names for the same type. For example, you might have two +or three local cluster setups you wish to test. + +### `include` + +```yaml: +include: + - +``` + +Allows including any number of other files. Similar to inheritance for +Docker Compose. The inheritance rules are: + +* Properties set later in the list take precedence over properties set in + files earlier in the list. +* Properties set in the file take precedence over properties set in + included files. +* Includes can nest to any level. + +Merging occurs as follows: + +* Top level scalars: newer values replace older values. +* Services: newer values replace all older settings for that service. +* Metastore init: newer values add more queries to any list defined + by an earlier file. +* Properties: newer values replace values defined by earlier files. + +The files are assumed to be resources (on the class path) and require +the full path name. Example: `/cluster/Commmon/base.yaml` + +### `proxyHost` + +```yaml +proxyHost: +``` + +When tests run in either Docker or Kubernetes, the test communicate with +a proxy, which forwards requests to the cluster hosts and ports. In +Docker, the proxy host is the machine that runs Docker. In Kubernetes, +the proxy host is the host running the Kubernetes proxy service. + +There is no proxy host for clusters running directly on a machine. + +If the proxy host is omitted for Docker, `localhost` is assumed. + +### `datasourceSuffix` + +```yaml +datasourceSuffix: +``` + +Suffix to append to data source names in indexer tests. The default +is the empty string. + +### `zk` + +```yaml +zk: + +``` + +Specifies the ZooKeeper instances. + +#### `startTimeoutSecs` + +```yaml +startTimeoutSecs: +``` + +Specifies the amount of time to wait for ZK to become available when using the +test client. Optional. + +### `metastore` + +```yaml +metastore: + +``` + +Describes the Druid "metadata storage" (metastore) typically +hosted in the offical MySql container. See `MetastoreConfig` for +configuration options. + +#### `driver` + +```yaml +driver: +``` + +The Driver to use to work with the metastore. The driver must be +available on the tests's class path. + +#### `connectURI` + +```yaml: +connectURI: +``` + +The JDBC connetion URL. Example: + +```text +jdbc:mysql://:/druid +``` + +The config system supports two special fields: `` and ``. +A string of form `` will be replaced by the resolved host name +(proxy host for Docker) and `` with the resolved port number. + +#### `user` + +```yaml +user: +``` + +The MySQL user name. + + +#### `password` + +```yaml +user: +``` + +The MySQL password. + +#### `properties` + +```yaml +properties: + : +``` + +Optional map of additional key/value pairs to pass to the JDBC driver. + +### `kafka` + +```yaml +zk: + +``` + +Describes the optional Kafka service. + +### `druid` + +```yaml +druid: + : + +``` + +Describes the set of Druid services using the `ServiceConfig` object. +Each service is keyed by the standard service name: the same name used +by the Druid `server` option. + +When using inheritance, overrides replace entire services: it is not possible +to override individual instances of the service. That is, an include file might +define `coordinator`, but a test-specific file might override this with a +definition of two Coordinators. + +### `properties` + +```yaml +properties: + : +``` + +Optional set of properties to use to configuration the Druid components loaded +by tests. This is the test-specific form of the standard Druid `common.runtime.properties` +and `runtime.properties` files. Because the test runs as a client, the server +files are not available, and might not even make sense. (The client is not +a "service", for example.) Technically, the properties listed here are added to +Guice as the one and only `Properties` object. + +Typically most components work using the default values. Tests are free to change +any of these values for a given test scenario. The properties are +the same for all tests within a category. However, they can be changed via environment +variables via the environment variable "binding" mechanism described in +[tests](tests.md). + +The "JSON configuration" mechanism wants all properties to be strings. YAML +will deserialize number-like properties as numbers. To avoid confusion, all +properties are converted to strings before being passed to Druid. + +When using inheritance, later properties override earlier properties. Environment +variables, if bound, override the defaults specified in this section. Command-line +settings, if provided, have the highest priority. + +A number of test-specific properties are avilable: + +* `druid.test.config.cloudBucket` +* `druid.test.config.cloudPath` + +### `settings` + +The settings section is much like the properties section, and, indeed, are converted +to properties internally. Settings are a fixed set of values that map to the config +files used in the prior tests. Keys include: + +| Setting | Property | Environment Variable | +| `druid_storage_type` | - | - | +| `druid_storage_bucket` | `druid.test.config.cloudBucket` | `DRUID_STORAGE_BUCKET` | +| `druid_storage_baseKey` | `druid.test.config.cloudPath` | `DRUID_STORAGE_BASEKEY` | +| `druid_s3_accessKey` | - | `AWS_ACCESS_KEY_ID` | +| `druid_s3_secretKey` | - | AWS_SECRET_ACCESS_KEY` | + +The above replaces the config file mechanism from the older tests. In general, when a +setting is fixed for a test category, list it in the `docker.yaml` configuration file. +When it varies, pass it in as an environment variable. As a result, the prior configuration +file is not needed. As a result, the prior `override.config.path` property is not supported. + +### `metastoreInit` + +```yaml +metastoreInit: + - sql: | + +``` + +A set of MySQL statements to be run against the +metadata storage before the test starts. Queries run in the +order specified. Ensure each is idempotent to +allow running tests multiple times against the same database. + +To be kind to readers, please format the statements across multiple lines. +The code will compress out extra spaces before submitting the query so +that JSON payloads are as compact as possible. + +The `sql` keyword is the only one supported at present. The idea is that +there may need to be context for some queries in some tests. (To be +enhanced as query conversion proceeds.) + +When using inheritance, the set of queries is the union of all queries +in all configuration files. Base statements appear first, then included +statements. + +### `metastoreInitDelaySec` + +```yaml +metastoreInitDelaySec: +``` + +The default value is 6 seconds. + +The metastore init section issues queries to the MySQL DB read by the +Coordinator. For performance, the Coordinator *does not* directly query +the database: instead, it queries an in-memory cache. This leads to the +following behavior: + +* The Coordinator starts, checks the DB, and records the poll time. +* The test starts and updates the DB. +* The test runs and issues a query that needs the DB contents. +* The Coordinator checks that its poll timeout has not yet occurred + and returns the (empty) contents of the cache. +* The test checks the empty contents against the expected contents, + notices the results differ, and fails the test. + +To work around this, we must change _two_ settings. First, change +the following Druid configuration for the Coordinator: + +```yaml + - druid_manager_segments_pollDuration=PT5S +``` + +Second, change the `metastoreInitDelaySec` to be a bit longer: + +```yaml +metastoreInitDelaySec: 6 +``` + +The result is that the test will sit idle for 6 seconds, but that is better +than random failures. + +**Note:** a better fix would be for the Coordinator to have an API that causes +it to flush its cache. Since some tests run two coordinators, the message must be +sent to both. An even better fix would be fore the Coordinator to detect such +changes itself somehow. + +### Service Object + +Generic object to describe Docker Compose services. + +#### `service` + +```yaml +service: +``` + +Name of the service as known to Docker Compose. Defaults to be +the same as the service name used in this configuration file. + +#### `instances` + +```yaml +instances: + - +``` + +Describes the instances of the service as `ServiceInstance` objects. +Each service requires at least one instance. If more than one, then +each instance must define a `tag` that is a suffix that distinguishes +the instances. + +### Service Instance Object + +The service sections all allow multiple instances of each service. Service +instances define each instance of a service and provide a number of properties: + +#### `tag` + +When a service has more than one instance, the instances must have unique +names. The name is made up of the a base name (see below) with the tag +appended. Thus, if the service is `cooordinator` and the tag is `one`, +then the instance name is `coordinator-one`. + +The tag is required when there is more than one instance of a service, +and is optional if there is only one instance. The tag corresponds to the +`DRUID_INSTANCE` environment variable passed into the container. + +#### `container` + +```yaml +container: +``` + +Name of the Docker container. If omitted, defaults to: + +* `-` if a `tag` is provided (see below.) +* The name of the service (if there is only one instance). + +#### `host` + +```yaml +host: +``` + +The host name or IP address on which the instance runs. This is +the host name known to the _cluster_: the name inside a Docker overlay network. +Has the same defaults as `container`. + +#### `port` + +```yaml +port: +``` + +The port number of the service on the container as seen by other +services running within Docker. Required. + +(TODO: If TLS is enabled, this is the TLS port.) + +#### `proxyPort` + +```yaml +proxyPort: +``` + +The port number for the service as exposed on the proxy host. +Defaults to the same as `port`. You must specify a value if +you run multiple instances of the same service. + +## Conversion Guide + +In prior tests, a config file, and the `ConfigFileConfigProvider` class, +provided test configuration. In this version, the file described here +provides configuration. This section presents a mapping from the old to +the new form. + +The `IntegrationTestingConfig` class, which the above class used to provide, +is reimplemented to provide the same information +to tests as before; only the source of the information has changed. + +The new framework assumes that each Druid node is configured either for +plain text or for TLS. (If this assumption is wrong, we'll change the config +file to match.) + +Many of the properties are derived from information in the configuration file. +For example, host names (within Docker) are those given in the `druid` section, +and ports (within the cluster and for the client) are given in `druid..intances.port`, +from which the code computes the URL. + +The old system hard-codes the idea that there are two coordinators or overlords. The +new system allows any number of instances. + +| Method | Old Property | New Format | +| ------ | ------------ | ---------- | +| Router | | | +| `getRouterHost()` | `router_host` | `'router'` | +| `getRouterUrl()` | `router_url` | `'router'` & `instances.port` | +| `getRouterTLSUrl()` | `router_tls_url` | " | +| `getPermissiveRouterUrl()` | `router_permissive_url` | " | +| `getPermissiveRouterTLSUrl()` | `router_permissive_tls_url` | " | +| `getNoClientAuthRouterUrl()` | `router_no_client_auth_url` | " | +| `getNoClientAuthRouterTLSUrl()` | `router_no_client_auth_tls_url` | " | +| `getCustomCertCheckRouterUrl()` | | " | +| `getCustomCertCheckRouterTLSUrl()` | | " | +| Broker | | | +| `getBrokerHost()` | `broker_host` | `'broker'` | +| `getBrokerUrl()` | `broker_url` | `'broker'` & `instances.port` | +| `getBrokerTLSUrl()` | `broker_tls_url` | " | +| Coordinator | | | +| `getCoordinatorHost()` | `coordinator_host` | `'coordinator'` + `tag` | +| `getCoordinatorTwoHost()` | `coordinator_two_host` | " | +| `getCoordinatorUrl()` | `coordinator_url` | host & `instances.port` | +| `getCoordinatorTLSUrl()` | `coordinator_tls_url` | " | +| `getCoordinatorTwoUrl()` | `coordinator_two_url` | " | +| `getCoordinatorTwoTLSUrl()` | `coordinator_two_tls_url` | " | +| Overlord | | | +| `getOverlordUrl()` | ? | `'overlord'` + `tag` | +| `getOverlordTwoHost()` | `overlord_two_host` | " | +| `getOverlordTwoUrl()` | `overlord_two_url` | host & `instances.port` | +| `getOverlordTLSUrl()` | ? | " | +| `getOverlordTwoTLSUrl()` | `overlord_two_tls_url` | " | +| Overlord | | | +| `getHistoricalHost()` | `historical_host` | `historical'` | +| `getHistoricalUrl()` | `historical_url` | `'historical'` & `instances.port` | +| `getHistoricalTLSUrl()` | `historical_tls_url` | " | +| Overlord | | | +| `getMiddleManagerHost()` | `middlemanager_host` | `'middlemanager'` | +| Dependencies | | | +| `getZookeeperHosts()` | `zookeeper_hosts` | `'zk'` | +| `getKafkaHost()` | `kafka_host` | '`kafka`' | +| `getSchemaRegistryHost()` | `schema_registry_host` | ? | +| `getProperty()` | From config file | From `settings` | +| `getProperties()` | " | " | +| `getUsername()` | `username` | Setting | +| `getPassword()` | `password` | Setting | +| `getCloudBucket()` | `cloud_bucket` | Setting | +| `getCloudPath()` | `cloud_path` | Setting | +| `getCloudRegion()` | `cloud_region` | Setting | +| `getS3AssumeRoleWithExternalId()` | `s3_assume_role_with_external_id` | Setting | +| `getS3AssumeRoleExternalId()` | `s3_assume_role_external_id` | Setting | +| `getS3AssumeRoleWithoutExternalId()` | `s3_assume_role_without_external_id` | Setting | +| `getAzureKey()` | `azureKey` | Setting | +| `getHadoopGcsCredentialsPath()` | `hadoopGcsCredentialsPath` | Setting | +| `getStreamEndpoint()` | `stream_endpoint` | Setting | +| `manageKafkaTopic()` | ? | ? | +| `getExtraDatasourceNameSuffix()` | ? | ? | + +Pre-defined environment bindings: + +| Setting | Env. Var. | +| `cloudBucket` | `DRUID_CLOUD_BUCKET` | +| `cloudPath` | `DRUID_CLOUD_PATH` | +| `s3AccessKey` | `AWS_ACCESS_KEY_ID` | +| `s3SecretKey` | `AWS_SECRET_ACCESS_KEY` | +| `azureContainer` | `AZURE_CONTAINER` | +| `azureAccount` | `AZURE_ACCOUNT` | +| `azureKey` | `AZURE_KEY` | +| `googleBucket` | `GOOGLE_BUCKET` | +| `googlePrefix` | `GOOGLE_PREFIX` | + +Others can be added in `Initializer.Builder`. + diff --git a/integration-tests-ex/docs/tests.md b/integration-tests-ex/docs/tests.md new file mode 100644 index 00000000000..1ef74a77338 --- /dev/null +++ b/integration-tests-ex/docs/tests.md @@ -0,0 +1,411 @@ + + +# Test Structure + +The structure of these integration tests is heavily influenced by the existing +integration test structure. In that previous structure: + +* Each test group ran as separate Maven build. +* Each would build an image, start a cluster, run the test, and shut down the cluster. +* Tests were created using [TestNG](https://testng.org/doc/), a long-obsolete + test framework. +* A `IntegrationTestingConfig` is created from system properties (passed in from + Maven via `-D=` options). +* A TestNG test runner uses a part of the Druid Guice configuration to inject + test objects into the tests. +* The test then runs. + +To minimize test changes, we try to keep much of the "interface" while changing +the "implementation". Basically: + +* The same Docker image is used for all tests. +* Each test defines its own test cluster using Docker Compose. +* Tests are grouped into categories, represented by [JUnit categories]( + https://junit.org/junit4/javadoc/4.12/org/junit/experimental/categories/Categories.html). +* Maven runs one selected category, starting and stopping the test-specific cluster + for each. +* A cluster-specific directory contains the `docker-compose.yaml` file that defines + that cluster. Each of these files imports from common definitions. +* Each test is annotated with the `DruidTestRunner` to handle initialization, and + JUnit `Category` to group the test into a category. +* Categories can share cluster configuration to reduce redundant definitions. +* A `docker.yaml` file defines the test configuration and creates the + `IntegrationTestingConfig` object. +* Tests run as JUnit tests. + +The remainder of this section describes the test internals. + +## Test Name + +Due to the way the [Failsafe]( +https://maven.apache.org/surefire/maven-failsafe-plugin/integration-test-mojo.html) +Maven plugin works, it will look for ITs with +names of the form "IT*.java". This is the preferred form for Druid ITs. That is, +name your test "ITSomething", not "SomethingTest" or "IntegTestSomething", etc. +Many tests are called "ITSomethingTest", but this is a bit repetitious and redundant +since "IT" stands for "Integration Test". + +## Cluster Configuration + +A test must have a [cluster configuration](compose.md) to define the cluster. +There is a many-to-one relationship between test categories and test clusters. + +## Test Configuration + +See [Test Configuration](test-config.md) for details on the `docker.yaml` file +that you create for each test module to tell the tests about the cluster you +have defined. + +Test configuration allows inheritance so, as in Docker Compose, we define +standard bits in one place, just providing test-specific information in each +tests `docker.yaml` file. + +The test code assumes that the test configuration file is in +`src/test/resources/cluster//docker.yaml`, where `` is +the test category. The test runner loads the configuration file into +(or, specifically that it is on the class path at `/yaml/docker.yaml`) +a `ClusterConfig` instance. + +The `ClusterConfig` instance provides the backward-compatible +`IntegrationTestingConfig` instance tha that most existing test cases use. +New tests may want to work with `ClusterConfig` directly as the older interface +is a bit of a muddle in several areas. + +## Test Category + +Each test is associated with a cluster definition. Maven starts the required +cluster, runs a group of tests, and shuts down the cluster. We use the JUnit +`Category` to identify the category for each test: + +```java +@RunWith(DruidTestRunner.class) +@Category(BatchIndex.class) +public class ITIndexerTest extends AbstractITBatchIndexTest +{ + ... +``` + +The category is a trivial class that exists just to provide the category name. +It can also hold annotations, which will use in a moment. When adding tests, use +and existing category, or define a new one if you want your tests to run in +parallel with other categories. + +The `test-cases` module contains all integration tests. However, +Maven can run only one category per Maven run. You specify the category using a +profile of the same name, but with "IT-" prefixed. Thus the Maven profile for the +above `BatchIndex` category is `IT-BatchIndex`. + +Test categories may share the same cluster definition. We mark this by adding an +annotation to the category (_not_ test) class. The test class itself: + +```java +@RunWith(DruidTestRunner.class) +@Category(InputFormat.class) +public class ITLocalInputSourceAllInputFormatTest extends AbstractLocalInputSourceParallelIndexTest +{ + ... +``` + +The test category class: + +```java +@Cluster(BatchIndex.class) +public class InputFormat +{ +} +``` + +This says that the test above is in the `InputFormat` category, and tests in that +category use the same cluster definition as the `BatchIndex` category. Specifically, +to look for the cluster definition in the `BatchIndex` folders. + +### Defined Categories + +At present, the following test categories are fully or partly converted: + +| Category | Test NG Group | Description | +| -------- | ------------- | ----------- | +| HighAvailability | high-availability | Cluster failover tests | +| BatchIndex | batch-index | Batch indexing tsets | +| InputFormat | input-format | Input format tests | + +The new names correspond to class names. The Test NG names were strings. + +## Test Runner + +The ITs are JUnit test, but use a special test runner to handle configuration. +Test configuration is complex. The easiest way to configure, once the configuration +files are set, is to use the `DruidTestRunner` class: + +```java +@RunWith(DruidTestRunner.class) +@Category(MyCategory.class) +public class MyTest +{ + @Inject + private SomeObject myObject; + ... + + @Test + public void myTest() + { + ... +``` + +The test runner loads the configuration files, configures Guice, starts the +Druid lifecycle, and injects the requested values into the class each time +a test method runs. For simple tests, this is all you need. + +The test runner validates that the test has a category, and handles the +above mapping from category to cluster definition. + +### Parameterization + +The `DruidTestRunner` extends `JUnitParamsRunner` to allow parameterized tests. +This class stays discretely out of the way if you don't care about parameters. +To use parameters, see the `CalciteJoinQueryTest` class for an example. + +## Initialization + +The JUnit-based integration tests are designed to be as simple as possible +to debug. Each test class uses annotations and configuration files to provide +all the information needed to run a test. Once the customer is started +(using `cluster.sh` as described [here](quickstart.md)), each test can +be run from the command line or IDE with no additional command-line parameters. +To do that, we use a `docker.yaml` configuration file that defines all needed +parameters, etc. + +A test needs both configuration and a Guice setup. The `DruidTestRunner` , +along with a number ofm support classes, mostly hide the details from the tests. +However, you should know what's being done so you can debug. + +* JUnit uses the annotation to notice that we've provided a custom + test runner. (When converting tests, remember to add the required + annotation.) +* JUnit calls the test class constructor one or more times per test class. +* On the first creation of the test class, `DruidTestRunner` creates an + instance of the `Initializer` class, via its `Builder` to + load test configuration, create the Guice injector, + inject dependencies into the class instanance, and + start the Druid lifecycle. +* JUnit calls one of the test methods in the class. +* On the second creation of the test class in the same JVM, `DruidTestRunner` + reuses the existing injector to inject dependencies into the test, + which avoids the large setup overhead. +* During the first configuration, `DruidTestRunner` causes initialization + to check the health of each service prior to starting the tests. +* The test is now configured just as it would be from TestNG, and is ready to run. +* `DruidTestRunner` ends the lifecycle after the last test within this class runs. + +See [this explanation](dependencies.md) for the gory details. + +`DruidTestRunner` loads the basic set of Druid modules to run the basic client +code. Tests may wish to load additional modules specific to that test. + +## Custom Configuration + +There are times when a test needs additional Guice modules beyond what the +`Initializer` provides. In such cases, you can add a method to customize +configuration. + +### Guice Modules + +If your test requires additional Guice modules, add them as follows: + +```java +@Configure +public static void configure(Initializer.Builder builder) +{ + builder.modules( + new MyExtraModule(), + new AnotherModule() + ); +} +``` + +### Properties + +Druid makes heavy use of properties to configure objects via the 'JsonConfigProvider` +mechanism. Integration tests don't read the usual `runtime.properties` files: there +is no such file to read. Instead, properties are set in the test configuration +file. There are times, however, when it makes more sense to hard-code a property +value. This is done in the `@Configure` method: + +```java + builder.property(key, value); +``` + +You can also bind a property to an environment variable. This value is used when +the environment variable is set. You should also bind a default value: + +```java + builder.property("druid.my.property", 42); + builder.propertyEnvVarBinding("druid.my.property", "ULTIMATE_ANSWER"); +``` + +A property can also be passed in as either a system property or an environment +variable of the "Docker property environment variable form": + +```bash +druid_property_a=foo +./it.sh Category test +``` + +Or, directly on the command line: + +```text +-Ddruid_property_b=bar +``` + +Property precedence is: + +* Properties set in code, as above. +* Properties from the configuration file. +* Properties bound to environment variables, and the environment variable is set. +* Properties from the command line. + +The test properties can also be seen as default values for properties provided +in config files or via the command line. + +## Resolving Lifecycle Issues + +If your test get the dreaded "it doesn't work that way" message, it means that +an injected property in your test is asking Guice to instantiate a lifecycle-managed +class after the lifecycle itself was started. This typically happens if the class +in question is bound via the polymorphic `PolyBind` mechanism which doesn't support +"eager singletons". (If the class in question is not created via `PolyBind`, change +its Guice binding to include `.asEagerSingleton()` rather than `.as(LazySingleton.class)`. +See [this reference](https://github.com/google/guice/wiki/Scopes#eager-singletons). + +A quick workaround is to tell the initializer to create an instance before the +lifecycle starts. The easy way to do that is simply to inject the object into a +field in your class. Otherwise, give the builder a hint: + +```java + builder.eagerInstance(ThePeskyComponent.class); +``` + +## Test Operation + +When working with tests, it is helpful to know a bit more about the "magic" +behind `DruidTestRunner`. + +Druid's code is designed to run in a server, not a client. Yet, the tests are +clients. This means that tests want to run code in a way that it was not +intended to be run. The existing ITs have mostly figured out how to make that +happen, but result is not very clean. This is an opportunity for improvement. + +Druid introduced a set of "injector builders" to organize Guice initialization +a bit. The builders normally build the full server Guice setup. For the ITs, +the builders also allow us to pick and choose which modules to use to define +a client. The `Initializer` class in `it-base` uses the injector builders to +define the "client" modules needed to run tests. + +Druid uses the `Lifecycle` class to start and stop services. For this to work, +the managed instance must be created *before* the lifecycle starts. There are +a few items that are lazy singletons. When run in the server, they work fine. +But, when run in tests, we run into a race condition: we want to start the +lifecycle once before the tests start, the inject dependencies into each test +class instance as tests run. But, those injections create the insteance we want +the lifecycle to manage, resulting in a muddle. This is why the `DruidTestRunner` +has that odd "first test. vs. subsequent test" logic. + +The prior ITs would start running tests immediately. But, it can take up to a +minute or more for a Druid cluster to stabilize as all the services start +running simultaneously. The previous ITs would use a generic retry up to 240 +times to work around the fact that any given test could fail due to the cluster +not being ready. This version does that startup check as part if `DruidTestRunner`. +By the time the tests run, the cluster is up and has reported itself healthy. +That is, your tests can assume a healthy cluster. If a test fails: it indicates +an actual error or race condition. + +Specifically, if tests still randomly fail, those tests are telling you something: something +in Druid itself is non-deterministic (such as the delay to see changes to the DB, etc.), +or the tests are making invalid assumptions such as assuming an ordering when there +is none, using a time delay to try to synchronize actions when there should be +some specific synchronization, etc. This means that, in general, you should avoid +the use of the generic retry facility: if you have to retry to get your tests to +work, then the Druid user has to also retry. Unless we document the need to retry +in the API documentation, then having to retry should be considered a bug to be fixed +(perhaps by documenting the need to retry, perhaps by fixing a bug, perhaps by adding +a synchronization API.) + +Another benefit of the startup check is that the startup and health-check costs are +paid once per test class. This allows you to structure your +tests as a large number of small tests rather than a few big tests. + +## `ClusterConfig` and `ResolvedClusterConfig` + +The `ClusterConfig` class is the Java representation of the +[test configuration](test-config.md). The instance is available from the +`Initializer` and by Guice injection. + +It is a Jackson-serialized class that handles the "raw" form of +configuration. + +The `ClusterConfig.resolve()` method expands includes, applies defaults, +validates values, and returns a `ResolvedClusterConfig` instance used +by tests. `ResolvedClusterConfig` is available via Guice injection. +In most cases, however, you'll use it indirecty via the various clients +described below. Each of those uses `IntegrationTestingConfig` class, an +instance of which is created to read from `ResolvedClusterConfig`. + +Remember that each host has two names and two ports: + +* The external (or "proxy") host and port, as seen by the machine running + the tests. +* The internal host and port, as seen by the service itself running + in the Docker cluster. + +The various [config files](test-config.md) provide configurations for +the Docker, K8s and local cluster cases. This means that `resolveProxyHost()` +will resolve to the proxy for Docker, but the actual host for a local cluster. + +The original test setup was designed before Druid introduced the router. +A good future improvement is to modify the code to use the router to do the +routing rather than doing it "by hand" in the tests. This means that each +test would use the router port and router API for things like the Overlord +and Coordinator. Then, configuration need only specify the router, not the +other services. + +It is also possible to use Router APIs to obtain the server list dynamically +rather than hard-coding the services and ports. If we find cases where tests +must use the APIs directly, then we could either extend the Router API or +implement client-side service lookup. + +## `ClusterClient` + +The integration tests make many REST calls to the Druid cluster. The tests +contain much copy/paste code to make these calls. The `ClusterClient` class +is intended to gather up these calls so we have a single implementation +rather than many copies. Add methods as needed for additional APIs. + +The cluster client is "test aware": it uses the information in +`ClusterConfig` to know how to send the requested API. The methods handle +JSON deserialization, so tests can focus simply on making a call and +checking the results. + +## `org.apache.druid.testing.clients` + +This package in `integration-tests` has clients for most other parts of +Druid. For example, `CoordinatorResourceTestClient` is a +client for Coordinator calls. These clients are also aware of the test +configuration, by way of the `IntegrationTestingConfig` class, an +instance of which is created to read from `ResolvedClusterConfig`. diff --git a/integration-tests-ex/docs/travis.md b/integration-tests-ex/docs/travis.md new file mode 100644 index 00000000000..f4b454b6df2 --- /dev/null +++ b/integration-tests-ex/docs/travis.md @@ -0,0 +1,168 @@ + + +# Travis Integration + +Apache Druid uses Travis to manage builds, including running the integration +tests. You can find the Travis build file at `$DRUID_DEV/.travis.yml`, where +`DRUID_DEV` is the root of your Druid development directory. Information +about Travis can be found at: + +* [Documentation](https://docs.travis-ci.com/) +* [Job lifecycle](https://docs.travis-ci.com/user/job-lifecycle/) +* [Environment variables](https://docs.travis-ci.com/user/environment-variables/) +* [Travis file reference](https://config.travis-ci.com/) +* [Travis YAML](https://docs.travis-ci.com/user/build-config-yaml) + +## Running ITs In Travis + +Travis integration is still experimental. The latest iteration is: + +```yaml + - name: "experimental docker tests" + stage: Tests - phase 1 + script: ${MVN} install -P test-image,docker-tests -rf :it-tools ${MAVEN_SKIP} -DskipUTs=true + after_failure: + - docker-tests/check-results.sh +``` + +The above is a Travis job definition. The job "inherits" an `install` task defined +earlier in the file. That install task builds all of Druid and creates the distribution +tarball. Since the tests are isolated in specialized Maven profiles, the `install` +task does not build any of the IT-related artifacts. + +We've placed the test run in "Phase 1" for debugging convenience. Later, the tests +will run in "Phase 2" along with the other ITs. Once conversion is complete, the +"previous generation" IT tests will be replaced by the newer revisions. + +The `script` runs the ITs. The components of the command line are: + +* `install` - Run Maven though the install [lifecycle phase]( + https://maven.apache.org/guides/introduction/introduction-to-the-lifecycle.html) + for each module. This allows us to build and install the "testing tools" + (see the [Maven notes](maven.md)). The test image is also built during the + `install` phase. The tests themselves only need the `verify` phase, which occurs + before `install`. `install` does nothing for ITs. +* `-P test-image,docker-tests` - activates the image to build the image + (`test-image`) and then runs the ITs (`docker-tests`). +* `-rf :it-tools` - The `it-tools` module is the first of the IT modules: it contains + the "testing tools" added into the image. Using `-rf` skips all the other projects + which we already built in the Travis `install` step. Doing so saves the time + otherwise required for Maven to figure out it has nothing to do for those modules. +* `${MAVEN_SKIP}` - Omits the static checks: they are not needed for ITs. +* `-DskipUTs=true` - The ITs use the [Maven Failsafe plugin]( + https://maven.apache.org/surefire/maven-failsafe-plugin/index.html) + which shares code with the [Maven Surefire plugin]( + https://maven.apache.org/surefire/maven-surefire-plugin/index.html). We don't want + to run unit tests. If we did the usual `-DskipTests`, then we'd also disable the + ITs. The `-DskipUTs=true` uses a bit of [Maven trickery]( + https://stackoverflow.com/questions/6612344/prevent-unit-tests-but-allow-integration-tests-in-maven) + to skip only the Surefire, but not Faisafe tests. + +## Travis Diagnostics + +A common failure when running ITs is that they uncover a bug in a Druid service; +typically in the code you added that you want to test. Or, if you are changing the +Docker or Docker Compose infratructure, then the tests will often fail because the +Druid services are mis-configured. (Bad configuration tends to result in services +that don't start, or start and immediately exit.) + +The standard way to diagnose such failures is to look at the Druid logs. However, +Travis provides no support for attaching files to a build. The best alternative +seems to be to upload the files somewhere else. As a compromise, the Travis build +will append to the build log a subset of the Druid logs. + +Travis has a limit of 4MB per build log, so we can't append the entire log for +every Druid service for every IT. We have to be selective. In most cases, we only +care about the logs for ITs that fail. + +Now, it turns out to be *very hard* indeed to capture failues! Eventually, we want +Maven to run many ITs for each test run: we need to know which failed. Each IT +creates its own "shared" directory, so to find the logs, we need to know which IT +failed. Travis does not have this information: Travis only knows that Maven itself +exited with a non-zero status. Maven doesn't know: it only knows that Failsafe +failed the build. Failsafe is designed to run all ITs, then check the results in +the `verify` state, so Maven doesn't even know about the failures. + +### Failsafe Error Reports + +To work around all this, we mimic Failsafe: we look at the Failsafe error report +in `$DRUID_DEV/docker-tests//target/failsafe-reports/failsafe-summary.xml` +which looks like this: + +```xml + + 3 + 1 + 0 + 0 + + +``` + +The above shows one error and no failures. A successful run will show 0 for the +`errors` tag. This example tells us "something didn't work". The corresponding +Druid service logs are candidates for review. + +### Druid Service Failures + +The Druid logs are in `$DRUID_DEV/docker-tests//target/shared/logs`. +We could append all of them, but recall the 4MB limit. We generally are +interested only in those services that failed. So, we look at the logs and +see that a successful run is indicated by a normal Lifecycle shutdown: + +```text +2022-04-16T20:54:37,997 INFO [Thread-56] org.apache.druid.java.util.common.lifecycle.Lifecycle - Stopping lifecycle [module] stage [INIT] +``` + +The key bit of text is: + +```text +Stopping lifecycle [module] stage [INIT] +``` + +This says that 1) we're shutting down the lifecycle (which means no exception was thrown), +and 2) that we got all the way to the end (`[INIT]`). Since Druid emits no final +"exited normally" message, we take the above as the next-best thing. + +So, we only care about logs that *don't* have the above line. For those, we want to +append the log to the build output. Or, because of the size limit, we append the +last 100 lines. + +All of this is encapsulated in the `docker-tests/check-results.sh` script which +is run if the build fails (in the `after_failure`) tag. + +### Druid Log Output + +For a failed test, the build log will end with something like this: + +```text +======= it-high-availability Failed ========== +broker.log logtail ======================== +022-04-16T03:53:10,492 INFO [CoordinatorRuleManager-Exec--0] org.apache.druid.discovery.DruidLeaderClient - Request[http://coordinator-one:8081/druid/coordinator/v1/rules] received redirect response to location [http://coordinator-two:8081/druid/coordinator/v1/rules]. +... +``` + +To keep below the limit, on the first failed test is reported. + +The above won't catch all cases: maybe the service exited normally, but might still have +log lines of interest. Since all tests run, those lines could be anywhere in the file +and the scripts can't know which might be of interest. To handle that, we either +have to upload all logs somewhere, or you can use the convenience of the new +IT framework to rerun the tests on your development machine. diff --git a/integration-tests-ex/it-image/build-image.sh b/integration-tests-ex/image/build-image.sh similarity index 100% rename from integration-tests-ex/it-image/build-image.sh rename to integration-tests-ex/image/build-image.sh diff --git a/integration-tests-ex/it-image/docker-build.sh b/integration-tests-ex/image/docker-build.sh similarity index 100% rename from integration-tests-ex/it-image/docker-build.sh rename to integration-tests-ex/image/docker-build.sh diff --git a/integration-tests-ex/it-image/docker/Dockerfile b/integration-tests-ex/image/docker/Dockerfile similarity index 100% rename from integration-tests-ex/it-image/docker/Dockerfile rename to integration-tests-ex/image/docker/Dockerfile diff --git a/integration-tests-ex/it-image/docker/druid.sh b/integration-tests-ex/image/docker/druid.sh similarity index 100% rename from integration-tests-ex/it-image/docker/druid.sh rename to integration-tests-ex/image/docker/druid.sh diff --git a/integration-tests-ex/it-image/docker/launch.sh b/integration-tests-ex/image/docker/launch.sh similarity index 76% rename from integration-tests-ex/it-image/docker/launch.sh rename to integration-tests-ex/image/docker/launch.sh index 4b8d0293c8c..1f64b4e14df 100644 --- a/integration-tests-ex/it-image/docker/launch.sh +++ b/integration-tests-ex/image/docker/launch.sh @@ -33,14 +33,38 @@ cd / # TODO: enable only for security-related tests? #/tls/generate-server-certs-and-keystores.sh -. /druid.sh # The image contains both the MySQL and MariaDB JDBC drivers. # The MySQL driver is selected by the Docker Compose file. # Set druid.metadata.mysql.driver.driverClassName to the preferred # driver. +# Test-specific way to define extensions. Compose defines two test-specific +# variables. We combine these to create the final form converted to a property. +if [ -n "$druid_extensions_loadList" ]; then + echo "Using the provided druid_extensions_loadList=$druid_extensions_loadList" +else + mkdir -p /tmp/conf + EXTNS_FILE=/tmp/conf/extns + echo $druid_standard_loadList | tr "," "\n" > $EXTNS_FILE + if [ -n "$druid_test_loadList" ]; then + echo $druid_test_loadList | tr "," "\n" >> $EXTNS_FILE + fi + druid_extensions_loadList="[" + delim="" + while read -r line; do + druid_extensions_loadList="$druid_extensions_loadList$delim\"$line\"" + delim="," + done < $EXTNS_FILE + export druid_extensions_loadList="${druid_extensions_loadList}]" + unset druid_standard_loadList + unset druid_test_loadList + rm $EXTNS_FILE + echo "Effective druid_extensions_loadList=$druid_extensions_loadList" +fi + # Create druid service config files with all the config variables +. /druid.sh setupConfig # Export the service config file path to use in supervisord conf file @@ -92,7 +116,9 @@ fi LOG_FILE=$LOG_DIR/${INSTANCE_NAME}.log echo "" >> $LOG_FILE -echo "--- Service runtime.properties ---" >> $LOG_FILE +echo "--- env ---" >> $LOG_FILE +env >> $LOG_FILE +echo "--- runtime.properties ---" >> $LOG_FILE cat $DRUID_SERVICE_CONF_DIR/*.properties >> $LOG_FILE echo "---" >> $LOG_FILE echo "" >> $LOG_FILE diff --git a/integration-tests-ex/it-image/docker/test-setup.sh b/integration-tests-ex/image/docker/test-setup.sh similarity index 100% rename from integration-tests-ex/it-image/docker/test-setup.sh rename to integration-tests-ex/image/docker/test-setup.sh diff --git a/integration-tests-ex/it-image/pom.xml b/integration-tests-ex/image/pom.xml similarity index 98% rename from integration-tests-ex/it-image/pom.xml rename to integration-tests-ex/image/pom.xml index cd0d4e4e272..e352aa41793 100644 --- a/integration-tests-ex/it-image/pom.xml +++ b/integration-tests-ex/image/pom.xml @@ -38,8 +38,9 @@ Reference: https://dzone.com/articles/build-docker-image-from-maven pom - it-image - it-image + org.apache.druid.integration-tests + druid-it-image + druid-it-image Build the Docker image for integration tests. @@ -199,15 +200,15 @@ Reference: https://dzone.com/articles/build-docker-image-from-maven - ${project.version} ${mysql.version} - ${mysql.image.version} ${mariadb.version} - ${apache.kafka.version} + com.mysql.jdbc.Driver + ${mysql.image.version} ${confluent-version} + ${apache.kafka.version} ${zookeeper.version} ${hadoop.compile.version} - com.mysql.jdbc.Driver + ${project.version} ${druid.it.image-name} ${project.build.directory} runtime diff --git a/integration-tests/src/main/java/org/apache/druid/cli/CustomNodeRoleCommandCreator.java b/integration-tests/src/main/java/org/apache/druid/cli/CustomNodeRoleCommandCreator.java index f2ac334b913..de8970b20ee 100644 --- a/integration-tests/src/main/java/org/apache/druid/cli/CustomNodeRoleCommandCreator.java +++ b/integration-tests/src/main/java/org/apache/druid/cli/CustomNodeRoleCommandCreator.java @@ -23,6 +23,7 @@ import com.github.rvesse.airline.builder.CliBuilder; public class CustomNodeRoleCommandCreator implements CliCommandCreator { + @SuppressWarnings({ "rawtypes", "unchecked" }) @Override public void addCommands(CliBuilder builder) { diff --git a/integration-tests/src/main/java/org/apache/druid/testing/ConfigFileConfigProvider.java b/integration-tests/src/main/java/org/apache/druid/testing/ConfigFileConfigProvider.java index e39c630cf28..8e3c5a83e8d 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/ConfigFileConfigProvider.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/ConfigFileConfigProvider.java @@ -221,7 +221,7 @@ public class ConfigFileConfigProvider implements IntegrationTestingConfigProvide overlordTwoTLSUrl = StringUtils.format("https://%s:%s", overlordTwoHost, props.get("overlord_two_tls_port")); } } - + middleManagerHost = props.get("middlemanager_host"); zookeeperHosts = props.get("zookeeper_hosts"); @@ -259,7 +259,6 @@ public class ConfigFileConfigProvider implements IntegrationTestingConfigProvide { return new IntegrationTestingConfig() { - @Override public String getCoordinatorUrl() { diff --git a/integration-tests/src/main/java/org/apache/druid/testing/IntegrationTestingConfigProvider.java b/integration-tests/src/main/java/org/apache/druid/testing/IntegrationTestingConfigProvider.java index 3ce1dd21f68..a815391fb22 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/IntegrationTestingConfigProvider.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/IntegrationTestingConfigProvider.java @@ -30,4 +30,5 @@ import com.google.inject.Provider; }) public interface IntegrationTestingConfigProvider extends Provider { + String PROPERTY_BASE = "druid.test.config"; } diff --git a/integration-tests/src/main/java/org/apache/druid/testing/clients/OverlordResourceTestClient.java b/integration-tests/src/main/java/org/apache/druid/testing/clients/OverlordResourceTestClient.java index 3df3d8a49d5..e3f2a98dc63 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/clients/OverlordResourceTestClient.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/clients/OverlordResourceTestClient.java @@ -323,6 +323,8 @@ public class OverlordResourceTestClient { TaskState status = getTaskStatus(taskID).getStatusCode(); if (status == TaskState.FAILED) { + LOG.error("Task failed: %s", taskID); + LOG.error("Message: %s", getTaskErrorMessage(taskID)); throw new ISE("Indexer task FAILED"); } return status == TaskState.SUCCESS; diff --git a/integration-tests/src/main/java/org/apache/druid/testing/guice/DruidTestModule.java b/integration-tests/src/main/java/org/apache/druid/testing/guice/DruidTestModule.java index 1e239ae57db..b0bef034083 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/guice/DruidTestModule.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/guice/DruidTestModule.java @@ -51,7 +51,7 @@ public class DruidTestModule implements Module binder.bind(IntegrationTestingConfig.class) .toProvider(IntegrationTestingConfigProvider.class) .in(ManageLifecycle.class); - JsonConfigProvider.bind(binder, "druid.test.config", IntegrationTestingConfigProvider.class); + JsonConfigProvider.bind(binder, IntegrationTestingConfigProvider.PROPERTY_BASE, IntegrationTestingConfigProvider.class); binder.bind(CuratorConfig.class).to(IntegrationTestingCuratorConfig.class); diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractIndexerTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractIndexerTest.java index 0c88619bc12..1bb6e8e12c8 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractIndexerTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractIndexerTest.java @@ -24,6 +24,7 @@ import com.google.inject.Inject; import org.apache.commons.io.IOUtils; import org.apache.druid.guice.annotations.Json; import org.apache.druid.guice.annotations.Smile; +import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.logger.Logger; @@ -165,6 +166,9 @@ public abstract class AbstractIndexerTest public static String getResourceAsString(String file) throws IOException { try (final InputStream inputStream = getResourceAsStream(file)) { + if (inputStream == null) { + throw new ISE("Failed to load resource: [%s]", file); + } return IOUtils.toString(inputStream, StandardCharsets.UTF_8); } } diff --git a/it.sh b/it.sh new file mode 100755 index 00000000000..6cf0b54a631 --- /dev/null +++ b/it.sh @@ -0,0 +1,130 @@ +#! /bin/bash + +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +#-------------------------------------------------------------------- + +# Utility script for running the new integration tests, since the Maven +# commands are unwieldy. + +export DRUID_DEV=$(cd $(dirname $0) && pwd) + +function usage +{ + cat < + start the cluster for category + down + stop the cluster for category + test + start the cluster, run the test for category, and stop the cluster + tail + show the last 20 lines of each container log + travis + run one IT in Travis (build dist, image, run test, tail logs) + prune + prune Docker volumes +EOF +} + +function tail_logs +{ + category=$1 + cd integration-tests-ex/cases/target/$category/logs + ls *.log | while read log; + do + echo "----- $category/$log -----" + tail -20 $log + done +} + +CMD=$1 +shift +MAVEN_IGNORE="-P skip-static-checks,skip-tests -Dmaven.javadoc.skip=true" + +case $CMD in + "help" ) + usage + ;; + "build" ) + mvn clean package -P dist $MAVEN_IGNORE -T1.0C + ;; + "dist" ) + mvn package -P dist $MAVEN_IGNORE -pl :distribution + ;; + "image" ) + cd $DRUID_DEV/integration-tests-ex/image + mvn install -P test-image $MAVEN_IGNORE + ;; + "up" ) + if [ -z "$1" ]; then + usage + exit 1 + fi + cd $DRUID_DEV/integration-tests-ex/cases + ./cluster.sh up $1 + ;; + "down" ) + if [ -z "$1" ]; then + usage + exit 1 + fi + cd $DRUID_DEV/integration-tests-ex/cases + ./cluster.sh down $1 + ;; + "test" ) + if [ -z "$1" ]; then + usage + exit 1 + fi + cd $DRUID_DEV/integration-tests-ex/cases + mvn verify -P skip-static-checks,docker-tests,IT-$1 \ + -Dmaven.javadoc.skip=true -DskipUTs=true \ + -pl :druid-it-cases + ;; + "tail" ) + if [ -z "$1" ]; then + usage + exit 1 + fi + tail_logs $1 + ;; + "travis" ) + if [ -z "$1" ]; then + usage + exit 1 + fi + $0 dist + $0 image + $0 test $1 + $0 tail $1 + ;; + "prune" ) + # Caution: this removes all volumes, which is generally what you + # want when testing. + docker system prune --volumes + ;; + * ) + usage + exit -1 + ;; +esac diff --git a/pom.xml b/pom.xml index 1eb1979a31b..7dede645166 100644 --- a/pom.xml +++ b/pom.xml @@ -214,8 +214,9 @@ distribution - integration-tests-ex/it-tools - integration-tests-ex/it-image + integration-tests-ex/tools + integration-tests-ex/image + integration-tests-ex/cases @@ -1205,7 +1206,8 @@ org/apache/druid/benchmark/**/* org/apache/druid/**/*Benchmark* org/apache/druid/testing/**/* - org/apache/druid/server/coordination/ServerManagerForQueryErrorTest.class org/apache/druid/guice/SleepModule.class @@ -1523,7 +1525,7 @@ org.apache.maven.plugins maven-surefire-plugin - 3.0.0-M6 + 3.0.0-M7 @@ -1878,10 +1880,11 @@ conf/** docker/*.conf docker/service-supervisords/*.conf - target/** + **/target/** licenses/** - **/test/resources/** - **/data/data/** + **/test/resources/**/* + **/docker/client_tls/* + resources/data/**/* **/derby.log **/jvm.config **/*.avsc @@ -1901,14 +1904,17 @@ .github/pull_request_template.md .github/dependabot.yml git.version - node_modules/** + website/node_modules/** src/**/*.snap examples/conf/** .asf.yaml **/dependency-reduced-pom.xml .editorconfig **/hadoop.indexer.libs.version - **/codegen/** + **/codegen/**/* + **/.settings/**/* + **/.classpath + **/.project diff --git a/server/src/main/java/org/apache/druid/guice/DruidInjectorBuilder.java b/server/src/main/java/org/apache/druid/guice/DruidInjectorBuilder.java index b9dcb7f3053..81e28431e2d 100644 --- a/server/src/main/java/org/apache/druid/guice/DruidInjectorBuilder.java +++ b/server/src/main/java/org/apache/druid/guice/DruidInjectorBuilder.java @@ -87,13 +87,27 @@ public class DruidInjectorBuilder /** * Add an arbitrary set of modules. - * - * @see #add(Object) */ public DruidInjectorBuilder add(Object...input) { for (Object o : input) { - add(o); + addInput(o); + } + return this; + } + + public DruidInjectorBuilder addModules(Module...inputs) + { + for (Object o : inputs) { + addInput(o); + } + return this; + } + + public DruidInjectorBuilder addAll(List inputs) + { + for (Object o : inputs) { + addInput(o); } return this; } @@ -106,11 +120,9 @@ public class DruidInjectorBuilder * modules have visibility only to objects defined in the base * injector, but not to objects defined in the injector being built. */ - public DruidInjectorBuilder add(Object input) + public DruidInjectorBuilder addInput(Object input) { - if (input instanceof DruidModule) { - return addDruidModule((DruidModule) input); - } else if (input instanceof Module) { + if (input instanceof Module) { return addModule((Module) input); } else if (input instanceof Class) { return addClass((Class) input); @@ -119,23 +131,15 @@ public class DruidInjectorBuilder } } - public DruidInjectorBuilder addDruidModule(DruidModule module) - { - if (!acceptModule(module.getClass())) { - return this; - } - baseInjector.injectMembers(module); - registerJacksonModules(module); - modules.add(module); - return this; - } - public DruidInjectorBuilder addModule(Module module) { if (!acceptModule(module.getClass())) { return this; } baseInjector.injectMembers(module); + if (module instanceof DruidModule) { + registerJacksonModules((DruidModule) module); + } modules.add(module); return this; } diff --git a/server/src/main/java/org/apache/druid/guice/MetadataConfigModule.java b/server/src/main/java/org/apache/druid/guice/MetadataConfigModule.java index ccff6dd2870..4a508826f2b 100644 --- a/server/src/main/java/org/apache/druid/guice/MetadataConfigModule.java +++ b/server/src/main/java/org/apache/druid/guice/MetadataConfigModule.java @@ -31,8 +31,8 @@ public class MetadataConfigModule implements Module @Override public void configure(Binder binder) { - JsonConfigProvider.bind(binder, "druid.metadata.storage.tables", MetadataStorageTablesConfig.class); - JsonConfigProvider.bind(binder, "druid.metadata.storage.connector", MetadataStorageConnectorConfig.class); + JsonConfigProvider.bind(binder, MetadataStorageTablesConfig.PROPERTY_BASE, MetadataStorageTablesConfig.class); + JsonConfigProvider.bind(binder, MetadataStorageConnectorConfig.PROPERTY_BASE, MetadataStorageConnectorConfig.class); JsonConfigProvider.bind(binder, "druid.manager.segments", SegmentsMetadataManagerConfig.class); JsonConfigProvider.bind(binder, "druid.manager.rules", MetadataRuleManagerConfig.class); diff --git a/server/src/main/java/org/apache/druid/initialization/ExtensionInjectorBuilder.java b/server/src/main/java/org/apache/druid/initialization/ExtensionInjectorBuilder.java index f030f62bc72..08614e2979f 100644 --- a/server/src/main/java/org/apache/druid/initialization/ExtensionInjectorBuilder.java +++ b/server/src/main/java/org/apache/druid/initialization/ExtensionInjectorBuilder.java @@ -39,7 +39,7 @@ public class ExtensionInjectorBuilder extends DruidInjectorBuilder this.serviceBuilder = serviceBuilder; ExtensionsLoader extnLoader = ExtensionsLoader.instance(baseInjector); for (DruidModule module : extnLoader.getFromExtensions(DruidModule.class)) { - addDruidModule(module); + addModule(module); } } diff --git a/services/src/main/java/org/apache/druid/cli/GuiceRunnable.java b/services/src/main/java/org/apache/druid/cli/GuiceRunnable.java index 9f694e45755..e20d64da51e 100644 --- a/services/src/main/java/org/apache/druid/cli/GuiceRunnable.java +++ b/services/src/main/java/org/apache/druid/cli/GuiceRunnable.java @@ -109,11 +109,11 @@ public abstract class GuiceRunnable implements Runnable } log.info( - "Starting up with processors[%,d], memory[%,d], maxMemory[%,d]%s. Properties follow.", + "Starting up with processors [%,d], memory [%,d], maxMemory [%,d]%s. Properties follow.", JvmUtils.getRuntimeInfo().getAvailableProcessors(), JvmUtils.getRuntimeInfo().getTotalHeapSizeBytes(), JvmUtils.getRuntimeInfo().getMaxHeapSizeBytes(), - directSizeBytes != null ? StringUtils.format(", directMemory[%,d]", directSizeBytes) : "" + directSizeBytes != null ? StringUtils.format(", directMemory [%,d]", directSizeBytes) : "" ); if (startupLoggingConfig.isLogProperties()) {