You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by GitBox <gi...@apache.org> on 2020/04/16 11:13:28 UTC

[GitHub] [druid] jon-wei opened a new pull request #9714: More Hadoop integration tests

jon-wei opened a new pull request #9714: More Hadoop integration tests
URL: https://github.com/apache/druid/pull/9714
 
 
   This PR adds a new set of Hadoop integration tests (not run in Travis CI in this PR).
   - Added Hadoop counterpart to the native batch wikipedia test, with variations on the partitioning spec. This reads from HDFS and writes to HDFS.
   - Added tests for Hadoop ingestion + cloud stores (Azure, GCS, S3).
     - Note that using Azure as deep storage with Hadoop 2.8.5 ingestion does not currently work due to conflicts in the `azure-storage` version that Druid uses (8.6.0) and the one that `hadoop-azure` uses (2.2.0)
   
   Some other changes included in this PR:
   - Enables the management proxy on the main router service, which makes it easier to troubleshoot the integration tests live using the web console
   - Removes an unneeded `-sha256` parameter from the `openssl genrsa` calls, which was causing problems in my environment with a newer openssl that rejects the unrecognized parameter
   - Allows setting `extra.datasource.name.suffix` for the integration tests, which is needed for these tests as our cloud store extensions currently don't handle the special characters in the datasource names properly.
   
   This PR has:
   - [x] been self-reviewed.
   - [ ] added documentation for new or modified features or behaviors.
   - [x] added Javadocs for most classes and all non-trivial methods. Linked related entities via Javadoc links.
   - [ ] added or updated version, license, or notice information in [licenses.yaml](https://github.com/apache/druid/blob/master/licenses.yaml)
   - [x] added comments explaining the "why" and the intent of the code wherever would not be obvious for an unfamiliar reader.
   - [ ] added unit tests or modified existing tests to cover new code paths.
   - [x] added integration tests.
   - [x] been tested in a test Druid cluster.
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] maytasm commented on a change in pull request #9714: More Hadoop integration tests

Posted by GitBox <gi...@apache.org>.
maytasm commented on a change in pull request #9714: More Hadoop integration tests
URL: https://github.com/apache/druid/pull/9714#discussion_r409919735
 
 

 ##########
 File path: integration-tests/src/test/java/org/apache/druid/tests/hadoop/ITGcsInputToHdfsHadoopIndexTest.java
 ##########
 @@ -0,0 +1,50 @@
+/*
+ * 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.tests.hadoop;
+
+import org.apache.druid.testing.guice.DruidTestModuleFactory;
+import org.apache.druid.tests.TestNGGroup;
+import org.testng.annotations.Guice;
+import org.testng.annotations.Test;
+
+/**
+ * IMPORTANT:
+ * To run this test, you must:
+ * 1) Set the bucket and path for your data. This can be done by setting -Ddruid.test.config.cloudBucket and
+ *    -Ddruid.test.config.cloudPath or setting "cloud_bucket" and "cloud_path" in the config file.
+ * 2. Set -Ddruid.test.config.hadoopGcsCredentialsPath to the location of your Google credentials file as it
+ *    exists within the Hadoop cluster that will ingest the data. The credentials file can be placed in the
+ *    shared folder used by the integration test containers if running the Docker-based Hadoop container,
+ *    in which case this property can be set to /shared/<path_of_your_credentials_file>
+ * 2) Copy wikipedia_index_data1.json, wikipedia_index_data2.json, and wikipedia_index_data3.json
+ *    located in integration-tests/src/test/resources/data/batch_index/json to your GCS at the location set in step 1.
+ * 3) Provide -Doverride.config.path=<PATH_TO_FILE> with gcs configs set. See
 
 Review comment:
   same as above

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] maytasm commented on a change in pull request #9714: More Hadoop integration tests

Posted by GitBox <gi...@apache.org>.
maytasm commented on a change in pull request #9714: More Hadoop integration tests
URL: https://github.com/apache/druid/pull/9714#discussion_r409913930
 
 

 ##########
 File path: integration-tests/run_cluster.sh
 ##########
 @@ -74,17 +74,18 @@
   # For druid-kinesis-indexing-service
   mkdir -p $SHARED_DIR/docker/extensions/druid-kinesis-indexing-service
   mv $SHARED_DIR/docker/lib/druid-kinesis-indexing-service-* $SHARED_DIR/docker/extensions/druid-kinesis-indexing-service
-  $ For druid-parquet-extensions
+  # For druid-parquet-extensions
   mkdir -p $SHARED_DIR/docker/extensions/druid-parquet-extensions
   mv $SHARED_DIR/docker/lib/druid-parquet-extensions-* $SHARED_DIR/docker/extensions/druid-parquet-extensions
-  $ For druid-orc-extensions
+  # For druid-orc-extensions
   mkdir -p $SHARED_DIR/docker/extensions/druid-orc-extensions
   mv $SHARED_DIR/docker/lib/druid-orc-extensions-* $SHARED_DIR/docker/extensions/druid-orc-extensions
 
   # Pull Hadoop dependency if needed
   if [ -n "$DRUID_INTEGRATION_TEST_START_HADOOP_DOCKER" ] && [ "$DRUID_INTEGRATION_TEST_START_HADOOP_DOCKER" == true ]
   then
-    java -cp "$SHARED_DIR/docker/lib/*" -Ddruid.extensions.hadoopDependenciesDir="$SHARED_DIR/hadoop-dependencies" org.apache.druid.cli.Main tools pull-deps -h org.apache.hadoop:hadoop-client:2.8.5 -h org.apache.hadoop:hadoop-aws:2.8.5
+    java -cp "$SHARED_DIR/docker/lib/*" -Ddruid.extensions.hadoopDependenciesDir="$SHARED_DIR/hadoop-dependencies" org.apache.druid.cli.Main tools pull-deps -h org.apache.hadoop:hadoop-client:2.8.5 -h org.apache.hadoop:hadoop-aws:2.8.5 -h org.apache.hadoop:hadoop-azure:2.8.5
+    curl https://storage.googleapis.com/hadoop-lib/gcs/gcs-connector-hadoop2-latest.jar --output $SHARED_DIR/docker/lib/gcs-connector-hadoop2-latest.jar
 
 Review comment:
   Can we not get gcs-connector-hadoop from pull-deps too?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] maytasm commented on a change in pull request #9714: More Hadoop integration tests

Posted by GitBox <gi...@apache.org>.
maytasm commented on a change in pull request #9714: More Hadoop integration tests
URL: https://github.com/apache/druid/pull/9714#discussion_r409920657
 
 

 ##########
 File path: integration-tests/src/test/java/org/apache/druid/tests/hadoop/ITHadoopIndexTest.java
 ##########
 @@ -19,87 +19,147 @@
 
 package org.apache.druid.tests.hadoop;
 
-import com.google.inject.Inject;
+import com.google.common.collect.ImmutableList;
+import org.apache.druid.indexer.partitions.DimensionBasedPartitionsSpec;
+import org.apache.druid.indexer.partitions.HashedPartitionsSpec;
+import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec;
 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.guice.DruidTestModuleFactory;
-import org.apache.druid.testing.utils.ITRetryUtil;
 import org.apache.druid.tests.TestNGGroup;
-import org.apache.druid.tests.indexer.AbstractIndexerTest;
-import org.testng.annotations.AfterClass;
-import org.testng.annotations.BeforeClass;
+import org.apache.druid.tests.indexer.AbstractITBatchIndexTest;
+import org.testng.annotations.DataProvider;
 import org.testng.annotations.Guice;
 import org.testng.annotations.Test;
 
+import java.io.Closeable;
+import java.util.UUID;
+import java.util.function.Function;
+
+/**
+ * IMPORTANT:
+ * To run this test, you must:
+ * 1) Copy wikipedia_index_data1.json, wikipedia_index_data2.json, and wikipedia_index_data3.json
+ *    located in integration-tests/src/test/resources/data/batch_index/json to your HDFS at the location set in step 1.
 
 Review comment:
   step 1 is this step. The location should be /resources/data/batch_index on the hadoop container fs and also at /batch_index on hdfs

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] maytasm commented on a change in pull request #9714: More Hadoop integration tests

Posted by GitBox <gi...@apache.org>.
maytasm commented on a change in pull request #9714: More Hadoop integration tests
URL: https://github.com/apache/druid/pull/9714#discussion_r409912713
 
 

 ##########
 File path: integration-tests/docker/environment-configs/override-examples/hadoop/gcs_to_gcs
 ##########
 @@ -0,0 +1,31 @@
+#
+# 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.
+#
+
+#
+# Example of override config file to provide.
+# Please replace <OVERRIDE_THIS> and <YOUR_GOOGLE_CREDENTIALS_FILE_NAME> with your cloud configs/credentials
+#
+druid_storage_type=google
+druid_google_bucket=<OVERRIDE_THIS>
+druid_google_prefix=<OVERRIDE_THIS>
+
+druid_extensions_loadList=["druid-google-extensions","druid-hdfs-storage"]
+
+# Set this to the path of the credentials file on the host where the integration tests are running
+GOOGLE_APPLICATION_CREDENTIALS=<OVERRIDE_THIS>
 
 Review comment:
   This should be the credentials file on the docker container.
   The run_cluster script will copy the files in the folder at -Dresource.file.dir.path to /shared/docker/credentials/
   So if you have the credentials file at /bob/folderx/secret.json then -Dresource.file.dir.path should be set to /bob/folderx/ and GOOGLE_APPLICATION_CREDENTIALS should be set to /shared/docker/credentials/secret.json

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] maytasm commented on a change in pull request #9714: More Hadoop integration tests

Posted by GitBox <gi...@apache.org>.
maytasm commented on a change in pull request #9714: More Hadoop integration tests
URL: https://github.com/apache/druid/pull/9714#discussion_r409922084
 
 

 ##########
 File path: integration-tests/src/test/java/org/apache/druid/tests/hadoop/ITHadoopIndexTest.java
 ##########
 @@ -19,87 +19,147 @@
 
 package org.apache.druid.tests.hadoop;
 
-import com.google.inject.Inject;
+import com.google.common.collect.ImmutableList;
+import org.apache.druid.indexer.partitions.DimensionBasedPartitionsSpec;
+import org.apache.druid.indexer.partitions.HashedPartitionsSpec;
+import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec;
 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.guice.DruidTestModuleFactory;
-import org.apache.druid.testing.utils.ITRetryUtil;
 import org.apache.druid.tests.TestNGGroup;
-import org.apache.druid.tests.indexer.AbstractIndexerTest;
-import org.testng.annotations.AfterClass;
-import org.testng.annotations.BeforeClass;
+import org.apache.druid.tests.indexer.AbstractITBatchIndexTest;
+import org.testng.annotations.DataProvider;
 import org.testng.annotations.Guice;
 import org.testng.annotations.Test;
 
+import java.io.Closeable;
+import java.util.UUID;
+import java.util.function.Function;
+
+/**
+ * IMPORTANT:
+ * To run this test, you must:
+ * 1) Copy wikipedia_index_data1.json, wikipedia_index_data2.json, and wikipedia_index_data3.json
+ *    located in integration-tests/src/test/resources/data/batch_index/json to your HDFS at the location set in step 1.
+ *    If using the Docker-based Hadoop container, this is automatically done by the integration tests.
+ * 2) Provide -Doverride.config.path=<PATH_TO_FILE> with HDFS configs set. See
+ *    integration-tests/docker/environment-configs/override-examples/hdfs for env vars to provide.
+ * 3) Run the test with -Dstart.hadoop.docker=true -Dextra.datasource.name.suffix='' in the mvn command
+ */
 @Test(groups = TestNGGroup.HADOOP_INDEX)
 @Guice(moduleFactory = DruidTestModuleFactory.class)
-public class ITHadoopIndexTest extends AbstractIndexerTest
+public class ITHadoopIndexTest extends AbstractITBatchIndexTest
 {
   private static final Logger LOG = new Logger(ITHadoopIndexTest.class);
+
   private static final String BATCH_TASK = "/hadoop/batch_hadoop_indexer.json";
   private static final String BATCH_QUERIES_RESOURCE = "/hadoop/batch_hadoop_queries.json";
   private static final String BATCH_DATASOURCE = "batchHadoop";
-  private boolean dataLoaded = false;
 
-  @Inject
-  private IntegrationTestingConfig config;
+  private static final String INDEX_TASK = "/hadoop/wikipedia_hadoop_index_task.json";
+  private static final String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_index_queries.json";
+  private static final String INDEX_DATASOURCE = "wikipedia_hadoop_index_test";
 
-  @BeforeClass
-  public void beforeClass()
-  {
-    loadData(config.getProperty("hadoopTestDir") + "/batchHadoop1");
-    dataLoaded = true;
-  }
+  private static final String REINDEX_TASK = "/hadoop/wikipedia_hadoop_reindex_task.json";
+  private static final String REINDEX_QUERIES_RESOURCE = "/indexer/wikipedia_reindex_queries.json";
+  private static final String REINDEX_DATASOURCE = "wikipedia_hadoop_reindex_test";
 
-  @Test
-  public void testHadoopIndex() throws Exception
+  @DataProvider
+  public static Object[][] resources()
   {
-    queryHelper.testQueriesFromFile(BATCH_QUERIES_RESOURCE, 2);
+    return new Object[][]{
+        {new HashedPartitionsSpec(3, null, null)},
+        {new HashedPartitionsSpec(null, 3, ImmutableList.of("page"))},
+        {new HashedPartitionsSpec(null, 3, ImmutableList.of("page", "user"))},
+        {new SingleDimensionPartitionsSpec(1000, null, null, false)},
+        {new SingleDimensionPartitionsSpec(1000, null, "page", false)},
+        {new SingleDimensionPartitionsSpec(1000, null, null, true)},
+
+        //{new HashedPartitionsSpec(null, 3, null)} // this results in a bug where the segments have 0 rows
+    };
   }
 
-  private void loadData(String hadoopDir)
+  @Test
+  public void testLegacyITHadoopIndexTest() throws Exception
   {
-    String indexerSpec;
+    try (
+        final Closeable ignored0 = unloader(BATCH_DATASOURCE + config.getExtraDatasourceNameSuffix());
+    ) {
+      final Function<String, String> specPathsTransform = spec -> {
+        try {
+          String path = "/batch_index/tsv";
 
 Review comment:
   currently /batch_index/tsv requires manual setup 
   From the integration-tests/README.md ... 
   ```
   Currently, ITHadoopIndexTest can only be run with your own Druid + Hadoop cluster by following the below steps:
   Create a directory called batchHadoop1 in the hadoop file system
   (anywhere you want) and put batch_hadoop.data (integration-tests/src/test/resources/hadoop/batch_hadoop.data) 
   into that directory (as its only file).
   ```
   We should automatically setup this dir for the hadoop docker container (similar to how we setup the wikipedia json files). You can create a new dir in integration-tests/src/test/resources/data/batch_index called tsv and copy integration-tests/src/test/resources/hadoop/batch_hadoop.data to integration-tests/src/test/resources/data/batch_index/tsv (the run-cluster script should handle the rest and create /batch_index/tsv with batch_hadoop.data inside)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] maytasm commented on a change in pull request #9714: More Hadoop integration tests

Posted by GitBox <gi...@apache.org>.
maytasm commented on a change in pull request #9714: More Hadoop integration tests
URL: https://github.com/apache/druid/pull/9714#discussion_r409911163
 
 

 ##########
 File path: integration-tests/docker/environment-configs/override-examples/hadoop/azure_to_hdfs
 ##########
 @@ -0,0 +1,34 @@
+#
+# 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.
+#
+
+#
+# Example of override config file to provide.
+# Please replace <OVERRIDE_THIS> with your cloud configs/credentials
+#
+druid_storage_type=hdfs
+druid_storage_storageDirectory=/druid/segments
+
+druid_extensions_loadList=["druid-azure-extensions","druid-hdfs-storage"]
+
+# Not used since we have HDFS deep storage, but the Druid Azure extension requires these to be defined
 
 Review comment:
   Not sure what do you mean by this? Doesn't reading the data from Azure still requires these?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] maytasm commented on a change in pull request #9714: More Hadoop integration tests

Posted by GitBox <gi...@apache.org>.
maytasm commented on a change in pull request #9714: More Hadoop integration tests
URL: https://github.com/apache/druid/pull/9714#discussion_r409913341
 
 

 ##########
 File path: integration-tests/docker/environment-configs/router
 ##########
 @@ -27,3 +27,5 @@ SERVICE_DRUID_JAVA_OPTS=-server -Xmx128m -XX:+UseG1GC -agentlib:jdwp=transport=d
 druid_auth_basic_common_cacheDirectory=/tmp/authCache/router
 druid_sql_avatica_enable=true
 druid_server_https_crlPath=/tls/revocations.crl
+druid_router_managementProxy_enabled=true
 
 Review comment:
   NIce!!!! 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] maytasm commented on a change in pull request #9714: More Hadoop integration tests

Posted by GitBox <gi...@apache.org>.
maytasm commented on a change in pull request #9714: More Hadoop integration tests
URL: https://github.com/apache/druid/pull/9714#discussion_r409926422
 
 

 ##########
 File path: integration-tests/src/test/resources/hadoop/wikipedia_hadoop_azure_input_index_task.json
 ##########
 @@ -0,0 +1,107 @@
+{
 
 Review comment:
   nit: Maybe we can combine the the different cloud input text task json into one by parameterizing the difference in the jobProperties. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] maytasm commented on a change in pull request #9714: More Hadoop integration tests

Posted by GitBox <gi...@apache.org>.
maytasm commented on a change in pull request #9714: More Hadoop integration tests
URL: https://github.com/apache/druid/pull/9714#discussion_r409932384
 
 

 ##########
 File path: integration-tests/src/test/java/org/apache/druid/tests/hadoop/ITHadoopIndexTest.java
 ##########
 @@ -19,87 +19,147 @@
 
 package org.apache.druid.tests.hadoop;
 
-import com.google.inject.Inject;
+import com.google.common.collect.ImmutableList;
+import org.apache.druid.indexer.partitions.DimensionBasedPartitionsSpec;
+import org.apache.druid.indexer.partitions.HashedPartitionsSpec;
+import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec;
 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.guice.DruidTestModuleFactory;
-import org.apache.druid.testing.utils.ITRetryUtil;
 import org.apache.druid.tests.TestNGGroup;
-import org.apache.druid.tests.indexer.AbstractIndexerTest;
-import org.testng.annotations.AfterClass;
-import org.testng.annotations.BeforeClass;
+import org.apache.druid.tests.indexer.AbstractITBatchIndexTest;
+import org.testng.annotations.DataProvider;
 import org.testng.annotations.Guice;
 import org.testng.annotations.Test;
 
+import java.io.Closeable;
+import java.util.UUID;
+import java.util.function.Function;
+
+/**
+ * IMPORTANT:
+ * To run this test, you must:
+ * 1) Copy wikipedia_index_data1.json, wikipedia_index_data2.json, and wikipedia_index_data3.json
+ *    located in integration-tests/src/test/resources/data/batch_index/json to your HDFS at the location set in step 1.
+ *    If using the Docker-based Hadoop container, this is automatically done by the integration tests.
+ * 2) Provide -Doverride.config.path=<PATH_TO_FILE> with HDFS configs set. See
+ *    integration-tests/docker/environment-configs/override-examples/hdfs for env vars to provide.
+ * 3) Run the test with -Dstart.hadoop.docker=true -Dextra.datasource.name.suffix='' in the mvn command
+ */
 @Test(groups = TestNGGroup.HADOOP_INDEX)
 @Guice(moduleFactory = DruidTestModuleFactory.class)
-public class ITHadoopIndexTest extends AbstractIndexerTest
+public class ITHadoopIndexTest extends AbstractITBatchIndexTest
 {
   private static final Logger LOG = new Logger(ITHadoopIndexTest.class);
+
   private static final String BATCH_TASK = "/hadoop/batch_hadoop_indexer.json";
   private static final String BATCH_QUERIES_RESOURCE = "/hadoop/batch_hadoop_queries.json";
   private static final String BATCH_DATASOURCE = "batchHadoop";
-  private boolean dataLoaded = false;
 
-  @Inject
-  private IntegrationTestingConfig config;
+  private static final String INDEX_TASK = "/hadoop/wikipedia_hadoop_index_task.json";
+  private static final String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_index_queries.json";
+  private static final String INDEX_DATASOURCE = "wikipedia_hadoop_index_test";
 
-  @BeforeClass
-  public void beforeClass()
-  {
-    loadData(config.getProperty("hadoopTestDir") + "/batchHadoop1");
-    dataLoaded = true;
-  }
+  private static final String REINDEX_TASK = "/hadoop/wikipedia_hadoop_reindex_task.json";
+  private static final String REINDEX_QUERIES_RESOURCE = "/indexer/wikipedia_reindex_queries.json";
+  private static final String REINDEX_DATASOURCE = "wikipedia_hadoop_reindex_test";
 
-  @Test
-  public void testHadoopIndex() throws Exception
+  @DataProvider
+  public static Object[][] resources()
   {
-    queryHelper.testQueriesFromFile(BATCH_QUERIES_RESOURCE, 2);
+    return new Object[][]{
+        {new HashedPartitionsSpec(3, null, null)},
+        {new HashedPartitionsSpec(null, 3, ImmutableList.of("page"))},
+        {new HashedPartitionsSpec(null, 3, ImmutableList.of("page", "user"))},
+        {new SingleDimensionPartitionsSpec(1000, null, null, false)},
+        {new SingleDimensionPartitionsSpec(1000, null, "page", false)},
+        {new SingleDimensionPartitionsSpec(1000, null, null, true)},
+
+        //{new HashedPartitionsSpec(null, 3, null)} // this results in a bug where the segments have 0 rows
+    };
   }
 
-  private void loadData(String hadoopDir)
+  @Test
+  public void testLegacyITHadoopIndexTest() throws Exception
   {
-    String indexerSpec;
+    try (
+        final Closeable ignored0 = unloader(BATCH_DATASOURCE + config.getExtraDatasourceNameSuffix());
+    ) {
+      final Function<String, String> specPathsTransform = spec -> {
+        try {
+          String path = "/batch_index/tsv";
 
 Review comment:
   Can you also update the integration-tests/README.md ITHadoopIndexTest can be run in hadoop docker container hence the existing README.md is out of date.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] maytasm commented on a change in pull request #9714: More Hadoop integration tests

Posted by GitBox <gi...@apache.org>.
maytasm commented on a change in pull request #9714: More Hadoop integration tests
URL: https://github.com/apache/druid/pull/9714#discussion_r409914159
 
 

 ##########
 File path: integration-tests/pom.xml
 ##########
 @@ -430,7 +431,7 @@
                                 -Dfile.encoding=UTF-8
                                 -Ddruid.test.config.dockerIp=${env.DOCKER_IP}
                                 -Ddruid.test.config.hadoopDir=${env.HADOOP_DIR}
-                                -Ddruid.test.config.extraDatasourceNameSuffix=\ Россия\ 한국\ 中国!?
+                                -Ddruid.test.config.extraDatasourceNameSuffix=${extra.datasource.name.suffix}
 
 Review comment:
   I believe there is a problem if using hadoop as deep storage and datasource contains special char / whitespace. Was that the reason for this change?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] maytasm commented on issue #9714: More Hadoop integration tests

Posted by GitBox <gi...@apache.org>.
maytasm commented on issue #9714: More Hadoop integration tests
URL: https://github.com/apache/druid/pull/9714#issuecomment-614975012
 
 
   Also, Do you think it's worth it to have the hdfs to cloud (i.e. ingesting /batch_index/tsv/*.data and using GCS/Azure/S3 as deep storage?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] maytasm commented on a change in pull request #9714: More Hadoop integration tests

Posted by GitBox <gi...@apache.org>.
maytasm commented on a change in pull request #9714: More Hadoop integration tests
URL: https://github.com/apache/druid/pull/9714#discussion_r409921208
 
 

 ##########
 File path: integration-tests/src/test/java/org/apache/druid/tests/hadoop/ITHadoopIndexTest.java
 ##########
 @@ -19,87 +19,147 @@
 
 package org.apache.druid.tests.hadoop;
 
-import com.google.inject.Inject;
+import com.google.common.collect.ImmutableList;
+import org.apache.druid.indexer.partitions.DimensionBasedPartitionsSpec;
+import org.apache.druid.indexer.partitions.HashedPartitionsSpec;
+import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec;
 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.guice.DruidTestModuleFactory;
-import org.apache.druid.testing.utils.ITRetryUtil;
 import org.apache.druid.tests.TestNGGroup;
-import org.apache.druid.tests.indexer.AbstractIndexerTest;
-import org.testng.annotations.AfterClass;
-import org.testng.annotations.BeforeClass;
+import org.apache.druid.tests.indexer.AbstractITBatchIndexTest;
+import org.testng.annotations.DataProvider;
 import org.testng.annotations.Guice;
 import org.testng.annotations.Test;
 
+import java.io.Closeable;
+import java.util.UUID;
+import java.util.function.Function;
+
+/**
+ * IMPORTANT:
+ * To run this test, you must:
+ * 1) Copy wikipedia_index_data1.json, wikipedia_index_data2.json, and wikipedia_index_data3.json
+ *    located in integration-tests/src/test/resources/data/batch_index/json to your HDFS at the location set in step 1.
+ *    If using the Docker-based Hadoop container, this is automatically done by the integration tests.
+ * 2) Provide -Doverride.config.path=<PATH_TO_FILE> with HDFS configs set. See
+ *    integration-tests/docker/environment-configs/override-examples/hdfs for env vars to provide.
+ * 3) Run the test with -Dstart.hadoop.docker=true -Dextra.datasource.name.suffix='' in the mvn command
+ */
 @Test(groups = TestNGGroup.HADOOP_INDEX)
 @Guice(moduleFactory = DruidTestModuleFactory.class)
-public class ITHadoopIndexTest extends AbstractIndexerTest
+public class ITHadoopIndexTest extends AbstractITBatchIndexTest
 
 Review comment:
   Maybe we should combine TestNGGroup.HADOOP_INDEX with TestNGGroup.HDFS_DEEP_STORAGE

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] maytasm commented on a change in pull request #9714: More Hadoop integration tests

Posted by GitBox <gi...@apache.org>.
maytasm commented on a change in pull request #9714: More Hadoop integration tests
URL: https://github.com/apache/druid/pull/9714#discussion_r409912772
 
 

 ##########
 File path: integration-tests/docker/environment-configs/override-examples/hadoop/gcs_to_hdfs
 ##########
 @@ -0,0 +1,31 @@
+#
+# 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.
+#
+
+#
+# Example of override config file to provide.
+# Please replace <OVERRIDE_THIS> and <YOUR_GOOGLE_CREDENTIALS_FILE_NAME> with your cloud configs/credentials
+#
+druid_storage_type=hdfs
+druid_storage_storageDirectory=/druid/segments
+
+druid_extensions_loadList=["druid-google-extensions","druid-hdfs-storage"]
+
+# Set this to the path of the credentials file on the host where the integration tests are running
+GOOGLE_APPLICATION_CREDENTIALS=<OVERRIDE_THIS>
 
 Review comment:
   same as above

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] maytasm commented on a change in pull request #9714: More Hadoop integration tests

Posted by GitBox <gi...@apache.org>.
maytasm commented on a change in pull request #9714: More Hadoop integration tests
URL: https://github.com/apache/druid/pull/9714#discussion_r409919515
 
 

 ##########
 File path: integration-tests/src/test/java/org/apache/druid/tests/hadoop/ITGcsInputToGcsHadoopIndexTest.java
 ##########
 @@ -0,0 +1,50 @@
+/*
+ * 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.tests.hadoop;
+
+import org.apache.druid.testing.guice.DruidTestModuleFactory;
+import org.apache.druid.tests.TestNGGroup;
+import org.testng.annotations.Guice;
+import org.testng.annotations.Test;
+
+/**
+ * IMPORTANT:
+ * To run this test, you must:
+ * 1) Set the bucket and path for your data. This can be done by setting -Ddruid.test.config.cloudBucket and
+ *    -Ddruid.test.config.cloudPath or setting "cloud_bucket" and "cloud_path" in the config file.
+ * 2. Set -Ddruid.test.config.hadoopGcsCredentialsPath to the location of your Google credentials file as it
+ *    exists within the Hadoop cluster that will ingest the data. The credentials file can be placed in the
 
 Review comment:
   Maybe worth mentioning about seting the -Dresource.file.dir.path and GOOGLE_APPLICATION_CREDENTIALS since you need to make druid-google-extensions  happy on the druid nodes. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] maytasm commented on a change in pull request #9714: More Hadoop integration tests

Posted by GitBox <gi...@apache.org>.
maytasm commented on a change in pull request #9714: More Hadoop integration tests
URL: https://github.com/apache/druid/pull/9714#discussion_r409931970
 
 

 ##########
 File path: integration-tests/src/test/java/org/apache/druid/tests/hadoop/ITHadoopIndexTest.java
 ##########
 @@ -19,87 +19,147 @@
 
 package org.apache.druid.tests.hadoop;
 
-import com.google.inject.Inject;
+import com.google.common.collect.ImmutableList;
+import org.apache.druid.indexer.partitions.DimensionBasedPartitionsSpec;
+import org.apache.druid.indexer.partitions.HashedPartitionsSpec;
+import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec;
 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.guice.DruidTestModuleFactory;
-import org.apache.druid.testing.utils.ITRetryUtil;
 import org.apache.druid.tests.TestNGGroup;
-import org.apache.druid.tests.indexer.AbstractIndexerTest;
-import org.testng.annotations.AfterClass;
-import org.testng.annotations.BeforeClass;
+import org.apache.druid.tests.indexer.AbstractITBatchIndexTest;
+import org.testng.annotations.DataProvider;
 import org.testng.annotations.Guice;
 import org.testng.annotations.Test;
 
+import java.io.Closeable;
+import java.util.UUID;
+import java.util.function.Function;
+
+/**
+ * IMPORTANT:
+ * To run this test, you must:
+ * 1) Copy wikipedia_index_data1.json, wikipedia_index_data2.json, and wikipedia_index_data3.json
+ *    located in integration-tests/src/test/resources/data/batch_index/json to your HDFS at the location set in step 1.
+ *    If using the Docker-based Hadoop container, this is automatically done by the integration tests.
+ * 2) Provide -Doverride.config.path=<PATH_TO_FILE> with HDFS configs set. See
+ *    integration-tests/docker/environment-configs/override-examples/hdfs for env vars to provide.
+ * 3) Run the test with -Dstart.hadoop.docker=true -Dextra.datasource.name.suffix='' in the mvn command
+ */
 @Test(groups = TestNGGroup.HADOOP_INDEX)
 @Guice(moduleFactory = DruidTestModuleFactory.class)
-public class ITHadoopIndexTest extends AbstractIndexerTest
+public class ITHadoopIndexTest extends AbstractITBatchIndexTest
 {
   private static final Logger LOG = new Logger(ITHadoopIndexTest.class);
+
   private static final String BATCH_TASK = "/hadoop/batch_hadoop_indexer.json";
   private static final String BATCH_QUERIES_RESOURCE = "/hadoop/batch_hadoop_queries.json";
   private static final String BATCH_DATASOURCE = "batchHadoop";
-  private boolean dataLoaded = false;
 
-  @Inject
-  private IntegrationTestingConfig config;
+  private static final String INDEX_TASK = "/hadoop/wikipedia_hadoop_index_task.json";
+  private static final String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_index_queries.json";
+  private static final String INDEX_DATASOURCE = "wikipedia_hadoop_index_test";
 
-  @BeforeClass
-  public void beforeClass()
-  {
-    loadData(config.getProperty("hadoopTestDir") + "/batchHadoop1");
-    dataLoaded = true;
-  }
+  private static final String REINDEX_TASK = "/hadoop/wikipedia_hadoop_reindex_task.json";
+  private static final String REINDEX_QUERIES_RESOURCE = "/indexer/wikipedia_reindex_queries.json";
+  private static final String REINDEX_DATASOURCE = "wikipedia_hadoop_reindex_test";
 
-  @Test
-  public void testHadoopIndex() throws Exception
+  @DataProvider
+  public static Object[][] resources()
   {
-    queryHelper.testQueriesFromFile(BATCH_QUERIES_RESOURCE, 2);
+    return new Object[][]{
+        {new HashedPartitionsSpec(3, null, null)},
+        {new HashedPartitionsSpec(null, 3, ImmutableList.of("page"))},
+        {new HashedPartitionsSpec(null, 3, ImmutableList.of("page", "user"))},
+        {new SingleDimensionPartitionsSpec(1000, null, null, false)},
+        {new SingleDimensionPartitionsSpec(1000, null, "page", false)},
+        {new SingleDimensionPartitionsSpec(1000, null, null, true)},
+
+        //{new HashedPartitionsSpec(null, 3, null)} // this results in a bug where the segments have 0 rows
+    };
   }
 
-  private void loadData(String hadoopDir)
+  @Test
+  public void testLegacyITHadoopIndexTest() throws Exception
   {
-    String indexerSpec;
+    try (
+        final Closeable ignored0 = unloader(BATCH_DATASOURCE + config.getExtraDatasourceNameSuffix());
+    ) {
+      final Function<String, String> specPathsTransform = spec -> {
+        try {
+          String path = "/batch_index/tsv";
 
 Review comment:
   Maybe also just get rid of the hadoopTestDir in the DockerConfigProvider. I think it's no longer needed. If using hadoop container then everything is automatically setup. If running your own hadoop then they should copy to /batch_index/tsv since the path is hardcoded in the specPathsTransform anyway

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] maytasm commented on a change in pull request #9714: More Hadoop integration tests

Posted by GitBox <gi...@apache.org>.
maytasm commented on a change in pull request #9714: More Hadoop integration tests
URL: https://github.com/apache/druid/pull/9714#discussion_r409924327
 
 

 ##########
 File path: integration-tests/src/test/java/org/apache/druid/tests/hadoop/ITS3InputToHdfsHadoopIndexTest.java
 ##########
 @@ -0,0 +1,49 @@
+/*
+ * 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.tests.hadoop;
+
+import org.apache.druid.testing.guice.DruidTestModuleFactory;
+import org.apache.druid.tests.TestNGGroup;
+import org.testng.annotations.Guice;
+import org.testng.annotations.Test;
+
+/**
+ * IMPORTANT:
+ * To run this test, you must:
+ * 1) Set the bucket, path, and region for your data.
+ *    This can be done by setting -Ddruid.test.config.cloudBucket, -Ddruid.test.config.cloudPath
+ *    and -Ddruid.test.config.cloudRegion or setting "cloud_bucket","cloud_path", and "cloud_region" in the config file.
+ * 2) Set -Ddruid.s3.accessKey and -Ddruid.s3.secretKey when running the tests to your access/secret keys.
+ * 3) Copy wikipedia_index_data1.json, wikipedia_index_data2.json, and wikipedia_index_data3.json
+ *    located in integration-tests/src/test/resources/data/batch_index/json to your S3 at the location set in step 1.
+ * 4) Provide -Doverride.config.path=<PATH_TO_FILE> with s3 credentials and hdfs deep storage configs set. See
+ *    integration-tests/docker/environment-configs/override-examples/hadoop/s3_to_hdfs for env vars to provide.
+ * 5) Run the test with -Dstart.hadoop.docker=true -Dextra.datasource.name.suffix='' in the mvn command
+ */
+@Test(groups = TestNGGroup.HADOOP_S3)
 
 Review comment:
   I think ITS3InputToHdfsHadoopIndexTest and ITS3InputToS3HadoopIndexTest should be in different groups. The groups can be the different deep storage. The reason for my suggestion is that if you run the whole group then you cannot switch  -Doverride.config.path between different test class. What will happens is that you will run the whole group (ITS3InputToHdfsHadoopIndexTest and ITS3InputToS3HadoopIndexTest) with the same druid config file which basically will be the exact same test (same deep storage). Same for the other cloud storages. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org