You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by "paul-rogers (via GitHub)" <gi...@apache.org> on 2023/02/17 20:09:30 UTC

[GitHub] [druid] paul-rogers commented on a diff in pull request #13774: Query tests migration to new IT framework with TLS

paul-rogers commented on code in PR #13774:
URL: https://github.com/apache/druid/pull/13774#discussion_r1110237022


##########
integration-tests-ex/cases/cluster/Common/environment-configs/test-groups/shuffle-deep-store:
##########
@@ -0,0 +1,23 @@
+#
+# 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.
+#
+
+# Test with deep storage as intermediate location to store shuffle data
+# Local deep storage will be used here
+druid_extensions_loadList=["mysql-metadata-storage","druid-basic-security","simple-client-sslcontext","druid-testing-tools","druid-lookups-cached-global","druid-histogram","druid-datasketches"]

Review Comment:
   The new ITs break the load list into multiple pieces, which are then combined in the container at runtime. I suspect this file was pulled from the old tests. Let's take a look at the `loadList` to see if we can convert it to the new, split format. (That is, only specify here those extensions which are not already given in the base env file.)



##########
integration-tests-ex/cases/cluster/Common/environment-configs/test-groups/custom-coordinator-duties:
##########
@@ -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_extensions_loadList=["druid-kafka-indexing-service","mysql-metadata-storage","druid-s3-extensions","druid-basic-security","simple-client-sslcontext","druid-testing-tools","druid-lookups-cached-global","druid-histogram","druid-datasketches"]

Review Comment:
   See note below about `loadList`.



##########
integration-tests-ex/cases/cluster/Common/environment-configs/historical-for-query-retry-error-test.env:
##########
@@ -0,0 +1,32 @@
+#
+# 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-for-query-error-test
+DRUID_LOG_PATH=/shared/logs/historical-for-query-error-test.log
+
+# JAVA OPTS
+DRUID_SERVICE_JAVA_OPTS=-server -Xmx512m -Xms512m -XX:+UseG1GC -agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=5010

Review Comment:
   The `agentlib` bit enables debugging. Does this test actually use the debugger as part of testing? Else, this might be left over from someone who had to do debugging and could thus be removed.



##########
integration-tests-ex/cases/src/test/resources/queries/broadcast_join_metadata_queries.json:
##########
@@ -0,0 +1,26 @@
+[
+  {
+    "description": "query information schema to make sure datasource is joinable and broadcast",
+    "query": {
+      "query": "SELECT TABLE_NAME, IS_JOINABLE, IS_BROADCAST FROM INFORMATION_SCHEMA.TABLES WHERE TABLE_NAME = '%%JOIN_DATASOURCE%%' AND IS_JOINABLE = 'YES' AND IS_BROADCAST = 'YES' AND TABLE_SCHEMA = 'druid'"
+    },
+    "expectedResults": [
+      {
+        "TABLE_NAME": "%%JOIN_DATASOURCE%%",
+        "IS_JOINABLE": "YES",
+        "IS_BROADCAST": "YES"
+      }
+    ]
+  },
+  {
+    "description": "query information schema to make sure druid schema is refreshed",
+    "query": {
+      "query": "SELECT COUNT(*) FROM INFORMATION_SCHEMA.COLUMNS WHERE TABLE_NAME = '%%JOIN_DATASOURCE%%'"
+    },
+    "expectedResults": [
+      {
+        "EXPR$0": 19
+      }
+    ]
+  }
+]

Review Comment:
   Nit: missing newline. Here and below.



##########
integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/query/ITBroadcastJoinQueryTest.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.query;
+
+import com.google.common.collect.ImmutableList;
+import com.google.inject.Inject;
+import org.apache.druid.curator.discovery.ServerDiscoveryFactory;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.io.Closer;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.java.util.http.client.HttpClient;
+import org.apache.druid.server.coordinator.rules.ForeverBroadcastDistributionRule;
+import org.apache.druid.testing.clients.CoordinatorResourceTestClient;
+import org.apache.druid.testing.guice.TestClient;
+import org.apache.druid.testing.utils.DataLoaderHelper;
+import org.apache.druid.testing.utils.ITRetryUtil;
+import org.apache.druid.testing.utils.SqlTestQueryHelper;
+import org.apache.druid.testsEx.categories.Query;
+import org.apache.druid.testsEx.config.DruidTestRunner;
+import org.apache.druid.testsEx.indexer.AbstractIndexerTest;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+
+@RunWith(DruidTestRunner.class)
+@Category(Query.class)
+public class ITBroadcastJoinQueryTest extends AbstractIndexerTest
+{
+  private static final Logger LOG = new Logger(ITBroadcastJoinQueryTest.class);
+  private static final String BROADCAST_JOIN_TASK = "/indexer/broadcast_join_index_task.json";
+  private static final String BROADCAST_JOIN_METADATA_QUERIES_RESOURCE = "/queries/broadcast_join_metadata_queries.json";
+  private static final String BROADCAST_JOIN_METADATA_QUERIES_AFTER_DROP_RESOURCE = "/queries/broadcast_join_after_drop_metadata_queries.json";
+  private static final String BROADCAST_JOIN_QUERIES_RESOURCE = "/queries/broadcast_join_queries.json";
+  private static final String BROADCAST_JOIN_DATASOURCE = "broadcast_join_wikipedia_test";
+
+
+  @Inject
+  ServerDiscoveryFactory factory;
+
+  @Inject
+  CoordinatorResourceTestClient coordinatorClient;
+
+  @Inject
+  SqlTestQueryHelper queryHelper;

Review Comment:
   This message seems to think that the superclass already has this item. If so, ensure that the item is protected (at least) and omit the copy here.



##########
integration-tests-ex/cases/cluster/Query/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.

Review Comment:
   This should be converted to the Python script format once I merge the PR that adds docker compose file generation.



##########
integration-tests-ex/cases/src/test/resources/queries/broadcast_join_after_drop_metadata_queries.json:
##########
@@ -0,0 +1,9 @@
+[
+  {
+    "description": "query information schema to make sure datasource is joinable and broadcast",
+    "query": {
+      "query": "SELECT TABLE_NAME, IS_JOINABLE, IS_BROADCAST FROM INFORMATION_SCHEMA.TABLES WHERE TABLE_NAME = '%%JOIN_DATASOURCE%%' AND IS_JOINABLE = 'YES' AND IS_BROADCAST = 'YES' AND TABLE_SCHEMA = 'druid'"
+    },
+    "expectedResults": []
+  }
+]

Review Comment:
   Nit: missing newline



##########
integration-tests-ex/cases/pom.xml:
##########
@@ -314,6 +314,32 @@
             <artifactId>curator-client</artifactId>
             <version>5.4.0</version>
         </dependency>
+        <dependency>
+            <groupId>org.apache.druid.extensions</groupId>
+            <artifactId>simple-client-sslcontext</artifactId>
+            <version>26.0.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>com.fasterxml.jackson.jaxrs</groupId>
+            <artifactId>jackson-jaxrs-smile-provider</artifactId>
+            <version>2.10.5</version>
+        </dependency>
+        <dependency>
+            <groupId>org.hamcrest</groupId>
+            <artifactId>hamcrest-core</artifactId>
+            <version>1.3</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.calcite.avatica</groupId>
+            <artifactId>avatica-core</artifactId>
+            <version>1.17.0</version>
+        </dependency>

Review Comment:
   Why might this test need Avatica?



##########
.travis.yml:
##########
@@ -543,6 +543,33 @@ jobs:
     # 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
+
+    - &integration_tests_ex
+      name: "(Compile=openjdk8, Run=openjdk11) query integration test (new)"
+      stage: Tests - phase 2
+      jdk: openjdk8
+      services: *integration_test_services
+      env: JVM_RUNTIME='-Djvm.runtime=11' USE_INDEXER='middleManager'
+      script: ./it.sh travis Query

Review Comment:
   No need to update the Travis file. Apache turned off Travis on the 15th. It is no longer available for Druid to use.



##########
integration-tests-ex/image/docker/tls/set-docker-host-ip.sh:
##########
@@ -0,0 +1,35 @@
+#!/bin/bash -eu
+
+# 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.

Review Comment:
   I wonder, were you able to work out what this script is trying to do? If so, can we add a header comment to explain it?
   
   On the surface, it seems we're trying to work out the IP address of the current host. The tests currently already use the loopback address. I wonder, did we find a case where that didn't work? It there something special about this set of tests that says what we're doing is to simple?
   
   If we set the host name here, then we probably want to do this at build time and write the value to `env.sh`. Or, we want to do this from `cluster.sh` before each Docker command. The tests also need the address, so they'd get it from the env var.
   
   Then, if we're computing this address, we should _not_ set it in the GHA scripts, which (I believe) we are doing today.
   
   If, however, we just copied this over because the old tests used it, we might consider not using the script if it is solving a problem we no longer actually have.



##########
integration-tests-ex/image/docker/tls/generate-server-certs-and-keystores.sh:
##########
@@ -0,0 +1,86 @@
+#!/bin/bash -eu
+
+# 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.

Review Comment:
   We've brought over the many security setup scripts from the old ITs, which is great. Thanks for doing it. But, I can't work out where they are actually called.
   
   It would seem that we need to call these when setting up the shared directory for a test. Are we doing so?



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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