From ca33d8bf2e131801318b0c170b7e312669443026 Mon Sep 17 00:00:00 2001 From: rishabh singh Date: Thu, 14 Mar 2024 16:49:41 +0530 Subject: [PATCH 01/31] Fix build --- .../org/apache/druid/sql/calcite/CalciteArraysQueryTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java index 141baa5e5308..2c165ffe3c3b 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java @@ -1160,7 +1160,7 @@ public void testArrayContainsArrayStringColumns() "SELECT ARRAY_CONTAINS(arrayStringNulls, ARRAY['a', 'b']), ARRAY_CONTAINS(arrayStringNulls, arrayString) FROM druid.arrays LIMIT 5", ImmutableList.of( newScanQueryBuilder() - .dataSource(DATA_SOURCE_ARRAYS) + .dataSource(CalciteTests.ARRAYS_DATASOURCE) .intervals(querySegmentSpec(Filtration.eternity())) .columns("v0", "v1") .virtualColumns( From 23da34a0b6eeb97e48d38db28108045a74badf14 Mon Sep 17 00:00:00 2001 From: rishabh singh Date: Mon, 15 Jul 2024 10:54:08 +0530 Subject: [PATCH 02/31] Test changes --- .../BackwardCompatibility/docker-compose.yaml | 106 ++++++++ .../cases/cluster/Common/dependencies.yaml | 6 +- integration-tests-ex/cases/pom.xml | 11 +- .../categories/BackwardCompatibility.java | 5 + .../druid/testsEx/indexer/ITIndexerTest.java | 3 +- .../leadership/HighAvailabilityTest.java | 241 ++++++++++++++++++ .../leadership/ITHighAvailabilityTest.java | 237 +---------------- .../cluster/BackwardCompatibility/docker.yaml | 40 +++ integration-tests-ex/image/build-image.sh | 2 + integration-tests-ex/image/docker-build.sh | 20 ++ integration-tests-ex/image/docker/Dockerfile | 5 +- integration-tests-ex/image/pom.xml | 4 + 12 files changed, 439 insertions(+), 241 deletions(-) create mode 100644 integration-tests-ex/cases/cluster/BackwardCompatibility/docker-compose.yaml create mode 100644 integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/categories/BackwardCompatibility.java create mode 100644 integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/leadership/HighAvailabilityTest.java create mode 100644 integration-tests-ex/cases/src/test/resources/cluster/BackwardCompatibility/docker.yaml diff --git a/integration-tests-ex/cases/cluster/BackwardCompatibility/docker-compose.yaml b/integration-tests-ex/cases/cluster/BackwardCompatibility/docker-compose.yaml new file mode 100644 index 000000000000..0b215462e082 --- /dev/null +++ b/integration-tests-ex/cases/cluster/BackwardCompatibility/docker-compose.yaml @@ -0,0 +1,106 @@ +# 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 + image: ${DRUID_PREV_IT_IMAGE_NAME} + 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 + + middlemanager: + extends: + file: ../Common/druid.yaml + service: middlemanager + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + volumes: + # Test data + - ../../resources:/resources + depends_on: + - zookeeper + + kafka: + extends: + file: ../Common/dependencies.yaml + service: kafka + depends_on: + - zookeeper diff --git a/integration-tests-ex/cases/cluster/Common/dependencies.yaml b/integration-tests-ex/cases/cluster/Common/dependencies.yaml index 0409c30bf538..0adf603ff91e 100644 --- a/integration-tests-ex/cases/cluster/Common/dependencies.yaml +++ b/integration-tests-ex/cases/cluster/Common/dependencies.yaml @@ -26,7 +26,7 @@ services: # See https://hub.docker.com/_/zookeeper zookeeper: # Uncomment the following when running on Apple Silicon processors: - # platform: linux/x86_64 + platform: linux/x86_64 image: zookeeper:${ZK_VERSION} container_name: zookeeper labels: @@ -46,7 +46,7 @@ services: kafka: image: bitnami/kafka:${KAFKA_VERSION} container_name: kafka - # platform: linux/x86_64 + platform: linux/x86_64 labels: druid-int-test: "true" ports: @@ -73,7 +73,7 @@ services: # The image will intialize the user and DB upon first start. metadata: # Uncomment the following when running on Apple Silicon processors: - # platform: linux/x86_64 + platform: linux/x86_64 image: mysql:$MYSQL_IMAGE_VERSION container_name: metadata labels: diff --git a/integration-tests-ex/cases/pom.xml b/integration-tests-ex/cases/pom.xml index 40461dd8ef17..26d170ed9562 100644 --- a/integration-tests-ex/cases/pom.xml +++ b/integration-tests-ex/cases/pom.xml @@ -459,6 +459,15 @@ GcsDeepStorage + + IT-BackwardCompatibility + + false + + + BackwardCompatibility + + docker-tests @@ -489,7 +498,7 @@ False - org.apache.druid.testsEx.categories.${it.category} + diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/categories/BackwardCompatibility.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/categories/BackwardCompatibility.java new file mode 100644 index 000000000000..8ce979fe4a2a --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/categories/BackwardCompatibility.java @@ -0,0 +1,5 @@ +package org.apache.druid.testsEx.categories; + +public class BackwardCompatibility +{ +} 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 index 65b8dc0b1ac0..542b89558fc2 100644 --- 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 @@ -27,6 +27,7 @@ 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.BackwardCompatibility; import org.apache.druid.testsEx.categories.BatchIndex; import org.apache.druid.testsEx.config.DruidTestRunner; import org.joda.time.Interval; @@ -43,7 +44,7 @@ import java.util.function.Function; @RunWith(DruidTestRunner.class) -@Category(BatchIndex.class) +@Category({BackwardCompatibility.class}) public class ITIndexerTest extends AbstractITBatchIndexTest { private static final String INDEX_TASK = "/indexer/wikipedia_index_task.json"; diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/leadership/HighAvailabilityTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/leadership/HighAvailabilityTest.java new file mode 100644 index 000000000000..765b9ac822ee --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/leadership/HighAvailabilityTest.java @@ -0,0 +1,241 @@ +/* + * 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.testsEx.categories.HighAvailability; +import org.apache.druid.testsEx.cluster.DruidClusterClient; +import org.apache.druid.testsEx.config.Initializer; +import org.apache.druid.testsEx.indexer.AbstractIndexerTest; +import org.apache.druid.testsEx.utils.DruidClusterAdminClient; +import org.junit.Test; + +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; + +public class HighAvailabilityTest +{ + private static final Logger LOG = new Logger(HighAvailabilityTest.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/leadership/ITHighAvailabilityTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/leadership/ITHighAvailabilityTest.java index 60d22f3f9f8f..42f633c3ff42 100644 --- 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 @@ -1,246 +1,13 @@ -/* - * 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.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.indexer.AbstractIndexerTest; -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 +public class ITHighAvailabilityTest extends HighAvailabilityTest { - 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/resources/cluster/BackwardCompatibility/docker.yaml b/integration-tests-ex/cases/src/test/resources/cluster/BackwardCompatibility/docker.yaml new file mode 100644 index 000000000000..d676f530e908 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/cluster/BackwardCompatibility/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/image/build-image.sh b/integration-tests-ex/image/build-image.sh index 4a31a7841944..51a8102d1cc3 100755 --- a/integration-tests-ex/image/build-image.sh +++ b/integration-tests-ex/image/build-image.sh @@ -43,6 +43,8 @@ export CONFLUENT_VERSION=$CONFLUENT_VERSION export MARIADB_VERSION=$MARIADB_VERSION export HADOOP_VERSION=$HADOOP_VERSION export DRUID_IT_IMAGE_NAME=$DRUID_IT_IMAGE_NAME +export DRUID_PREV_VERSION=$DRUID_PREV_VERSION +export DRUID_PREV_IT_IMAGE_NAME=$DRUID_PREV_IT_IMAGE_NAME EOF exec bash $SCRIPT_DIR/docker-build.sh diff --git a/integration-tests-ex/image/docker-build.sh b/integration-tests-ex/image/docker-build.sh index 6a945aa6129a..e27280b2d187 100755 --- a/integration-tests-ex/image/docker-build.sh +++ b/integration-tests-ex/image/docker-build.sh @@ -26,6 +26,8 @@ SCRIPT_DIR=$(cd $(dirname $0) && pwd) +echo "script_dir is $SCRIPT_DIR" + # Maven should have created the docker dir with the needed # dependency jars. If doing this by hand, run Maven once to # populate these jars. @@ -39,6 +41,8 @@ mkdir -p $TARGET_DIR/docker cp -r docker/* $TARGET_DIR/docker cd $TARGET_DIR/docker +echo "target dir is $TARGET_DIR" + # Grab the distribution if needed (skipped if no change.) DISTRIB_FILE=apache-druid-$DRUID_VERSION-bin.tar.gz SOURCE_FILE=$PARENT_DIR/distribution/target/$DISTRIB_FILE @@ -46,6 +50,11 @@ if [[ ! -f $DISTRIB_FILE || $SOURCE_FILE -nt $DISTRIB_FILE ]]; then cp $SOURCE_FILE . fi +# Download the previous druid tar +curl -L https://dlcdn.apache.org/druid/30.0.0/apache-druid-30.0.0-bin.tar.gz --output apache-druid-30.0.0-bin.tar.gz + +echo "yaya download done" + docker build -t $DRUID_IT_IMAGE_NAME \ --build-arg DRUID_VERSION=$DRUID_VERSION \ --build-arg MYSQL_VERSION=$MYSQL_VERSION \ @@ -53,4 +62,15 @@ docker build -t $DRUID_IT_IMAGE_NAME \ --build-arg CONFLUENT_VERSION=$CONFLUENT_VERSION \ --build-arg HADOOP_VERSION=$HADOOP_VERSION \ --build-arg MYSQL_DRIVER_CLASSNAME=$MYSQL_DRIVER_CLASSNAME \ + --build-arg DRUID_TESTING_TOOLS_VERSION=$DRUID_VERSION \ + . + +docker build -t $DRUID_PREV_IT_IMAGE_NAME \ + --build-arg DRUID_VERSION=$DRUID_PREV_VERSION \ + --build-arg MYSQL_VERSION=$MYSQL_VERSION \ + --build-arg MARIADB_VERSION=$MARIADB_VERSION \ + --build-arg CONFLUENT_VERSION=$CONFLUENT_VERSION \ + --build-arg HADOOP_VERSION=$HADOOP_VERSION \ + --build-arg MYSQL_DRIVER_CLASSNAME=$MYSQL_DRIVER_CLASSNAME \ + --build-arg DRUID_TESTING_TOOLS_VERSION=$DRUID_VERSION \ . diff --git a/integration-tests-ex/image/docker/Dockerfile b/integration-tests-ex/image/docker/Dockerfile index a77a5c2d023e..99b2da9476b2 100644 --- a/integration-tests-ex/image/docker/Dockerfile +++ b/integration-tests-ex/image/docker/Dockerfile @@ -46,13 +46,15 @@ ARG MARIADB_VERSION ENV MARIADB_VERSION=$MARIADB_VERSION ARG MYSQL_DRIVER_CLASSNAME=com.mysql.jdbc.Driver ENV MYSQL_DRIVER_CLASSNAME=$MYSQL_DRIVER_CLASSNAME +ARG DRUID_TESTING_TOOLS_VERSION +ENV DRUID_TESTING_TOOLS_VERSION=$DRUID_TESTING_TOOLS_VERSION ENV DRUID_HOME=/usr/local/druid # Populate build artifacts COPY apache-druid-${DRUID_VERSION}-bin.tar.gz /usr/local/ -COPY druid-it-tools-${DRUID_VERSION}.jar /tmp/druid/extensions/druid-it-tools/ +COPY druid-it-tools-${DRUID_TESTING_TOOLS_VERSION}.jar /tmp/druid/extensions/druid-it-tools/ COPY kafka-protobuf-provider-${CONFLUENT_VERSION}.jar /tmp/druid/lib/ COPY mysql-connector-j-${MYSQL_VERSION}.jar /tmp/druid/lib/ COPY mariadb-java-client-${MARIADB_VERSION}.jar /tmp/druid/lib/ @@ -71,3 +73,4 @@ RUN bash /test-setup.sh USER druid:druid WORKDIR / ENTRYPOINT [ "bash", "/launch.sh" ] + diff --git a/integration-tests-ex/image/pom.xml b/integration-tests-ex/image/pom.xml index 600501edb92b..f359c14dc3a2 100644 --- a/integration-tests-ex/image/pom.xml +++ b/integration-tests-ex/image/pom.xml @@ -55,6 +55,8 @@ Reference: https://dzone.com/articles/build-docker-image-from-maven image name will typically be provided by the build enfironment, and will override this default name. --> ${project.groupId}/test:${project.version} + 30.0.0 + ${project.groupId}/test:${druid.it.prev.version} 5.5.1 2.7.3 5.7-debian @@ -209,6 +211,8 @@ Reference: https://dzone.com/articles/build-docker-image-from-maven ${hadoop.compile.version} ${project.version} ${druid.it.image-name} + ${druid.it.prev.version} + ${druid.it.prev.image-name} ${project.build.directory} From cef932311bf89225f53f802288f97a03d6a28963 Mon Sep 17 00:00:00 2001 From: rishabh singh Date: Mon, 15 Jul 2024 10:55:58 +0530 Subject: [PATCH 03/31] Minor change --- integration-tests-ex/image/docker-build.sh | 4 ---- 1 file changed, 4 deletions(-) diff --git a/integration-tests-ex/image/docker-build.sh b/integration-tests-ex/image/docker-build.sh index e27280b2d187..dfbc0fe84531 100755 --- a/integration-tests-ex/image/docker-build.sh +++ b/integration-tests-ex/image/docker-build.sh @@ -41,8 +41,6 @@ mkdir -p $TARGET_DIR/docker cp -r docker/* $TARGET_DIR/docker cd $TARGET_DIR/docker -echo "target dir is $TARGET_DIR" - # Grab the distribution if needed (skipped if no change.) DISTRIB_FILE=apache-druid-$DRUID_VERSION-bin.tar.gz SOURCE_FILE=$PARENT_DIR/distribution/target/$DISTRIB_FILE @@ -53,8 +51,6 @@ fi # Download the previous druid tar curl -L https://dlcdn.apache.org/druid/30.0.0/apache-druid-30.0.0-bin.tar.gz --output apache-druid-30.0.0-bin.tar.gz -echo "yaya download done" - docker build -t $DRUID_IT_IMAGE_NAME \ --build-arg DRUID_VERSION=$DRUID_VERSION \ --build-arg MYSQL_VERSION=$MYSQL_VERSION \ From 5d3aa91fdf63131ee88c966a9eabe794e40e7eb7 Mon Sep 17 00:00:00 2001 From: rishabh singh Date: Tue, 16 Jul 2024 22:06:33 +0530 Subject: [PATCH 04/31] GHA changes --- .github/workflows/file1.yml | 16 +++ .github/workflows/file2.yml | 20 ++++ .github/workflows/file3.yml | 100 ++++++++++++++++++ .github/workflows/reusable-revised-its.yml | 34 ++++++ .github/workflows/revised-its.yml | 29 ++++- .../unit-and-integration-tests-unified.yml | 25 ++++- docs/ingestion/input-sources.md | 8 +- .../BackwardCompatibility/docker-compose.yaml | 3 +- .../druid/testsEx/indexer/ITIndexerTest.java | 1 + integration-tests-ex/image/build-image.sh | 3 +- integration-tests-ex/image/docker-build.sh | 14 ++- integration-tests-ex/image/pom.xml | 6 +- 12 files changed, 241 insertions(+), 18 deletions(-) create mode 100644 .github/workflows/file1.yml create mode 100644 .github/workflows/file2.yml create mode 100644 .github/workflows/file3.yml diff --git a/.github/workflows/file1.yml b/.github/workflows/file1.yml new file mode 100644 index 000000000000..16812b1ff729 --- /dev/null +++ b/.github/workflows/file1.yml @@ -0,0 +1,16 @@ + +name: "A1" + +env: + MYSQL_DRIVER_CLASSNAME: com.mysql.jdbc.Driver # Used to set druid config in docker image for revised ITs + SEGMENT_DOWNLOAD_TIMEOUT_MINS: 5 + PREVIOUS_DRUID_VERSIONS: '["30.0.0", "29.0.0"]' + TEST_VAR: 2 + + +jobs: + mop: + if: true + uses: ./.github/workflows/file2.yml + with: + previous_versions: ${{ env.PREVIOUS_DRUID_VERSIONS }} \ No newline at end of file diff --git a/.github/workflows/file2.yml b/.github/workflows/file2.yml new file mode 100644 index 000000000000..4be4773d85d9 --- /dev/null +++ b/.github/workflows/file2.yml @@ -0,0 +1,20 @@ + +name: "A2" +on: + workflow_call: + inputs: + previous_versions: + description: "Previous druid versions to run the test against." + required: true + type: string + workflow_dispatch: + +jobs: + echo_array_var_json: + runs-on: ubuntu-latest # You can choose a different runner if needed + steps: + - name: Echo environment variable (array as JSON) + run: | + echo $TEST_VAR + echo "printing var" + echo "${{inputs.previous_versions}}" \ No newline at end of file diff --git a/.github/workflows/file3.yml b/.github/workflows/file3.yml new file mode 100644 index 000000000000..be9d86460cb7 --- /dev/null +++ b/.github/workflows/file3.yml @@ -0,0 +1,100 @@ +# 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. + +name: "Xyz" + +env: + MYSQL_DRIVER_CLASSNAME: com.mysql.jdbc.Driver # Used to set druid config in docker image for revised ITs + SEGMENT_DOWNLOAD_TIMEOUT_MINS: 5 + BACKWARD_COMPATIBILITY_IT_ENABLED: false + DRUID_PREVIOUS_VERSION: 30.0.0 + DRUID_PREVIOUS_VERSION_DOWNLOAD_URL: https://dlcdn.apache.org/druid/30.0.0/apache-druid-30.0.0-bin.tar.gz + DRUID_PREVIOUS_IT_IMAGE_NAME: org.apache.druid.integration-tests/test:30.0.0 + +jobs: + test-build: + name: "build" + runs-on: ubuntu-latest + steps: + - name: Checkout branch + uses: actions/checkout@v4 + + # skip the "cache: maven" step from setup-java. We explicitly use a + # different cache key since we cannot reuse it across commits. + - uses: actions/setup-java@v4 + with: + distribution: 'zulu' + java-version: 8 + + # the build step produces SNAPSHOT artifacts into the local maven repository, + # we include github.sha in the cache key to make it specific to that build/jdk + - name: Cache Maven m2 repository + id: maven + uses: actions/cache@v4 + with: + path: ~/.m2/repository + key: maven-${{ runner.os }}-8-${{ github.sha }} + restore-keys: setup-java-Linux-maven-${{ hashFiles('**/pom.xml') }} + + - name: Cache targets + id: target + uses: actions/cache@v4 + with: + path: | + ./**/target + key: maven-${{ runner.os }}-8-targets-${{ github.sha }} + + - name: Cache image + id: docker_container + uses: actions/cache@v4 + with: + key: druid-container-jdk8.tar.gz-${{ github.sha }} + path: | + ./druid-container-jdk8.tar.gz + ./integration-tests-ex/image/target/env.sh + + - name: Maven build + id: maven_build + run: | + ./it.sh ci + + - name: Container build + run: | + ./it.sh image + source ./integration-tests-ex/image/target/env.sh + docker tag $DRUID_IT_IMAGE_NAME $DRUID_IT_IMAGE_NAME-jdk8 + + - name: Save docker container to archive + run: | + source ./integration-tests-ex/image/target/env.sh + echo $DRUID_IT_IMAGE_NAME + docker save "$DRUID_IT_IMAGE_NAME" | gzip > druid-container-jdk8.tar.gz + + - name: Save old version docker image + if: (${{ env.BACKWARD_COMPATIBILITY_IT_ENABLED == 'true' }} + run: | + docker tag $DRUID_PREVIOUS_IT_IMAGE_NAME $DRUID_PREVIOUS_IT_IMAGE_NAME-jdk8-version${{ inputs.DRUID_PREVIOUS_VERSION }} + echo $DRUID_PREVIOUS_IT_IMAGE_NAME + docker save "$DRUID_PREVIOUS_IT_IMAGE_NAME" | gzip > druid-container-jdk8-version${{ inputs.DRUID_PREVIOUS_VERSION }}.tar.gz + + test-revised-its: + needs: test-build + if: ${{ always() }} + uses: ./.github/workflows/revised-its.yml + with: + BACKWARD_COMPATIBILITY_IT_ENABLED: ${{ env.BACKWARD_COMPATIBILITY_IT_ENABLED }} + DRUID_PREVIOUS_VERSION: ${{ env.DRUID_PREVIOUS_VERSION }} + DRUID_PREVIOUS_VERSION_DOWNLOAD_URL: ${{ env.DRUID_PREVIOUS_VERSION_DOWNLOAD_URL }} + DRUID_PREVIOUS_IT_IMAGE_NAME: ${{ env.DRUID_PREVIOUS_IT_IMAGE_NAME }} diff --git a/.github/workflows/reusable-revised-its.yml b/.github/workflows/reusable-revised-its.yml index d9237a52abab..0353ece0b7f8 100644 --- a/.github/workflows/reusable-revised-its.yml +++ b/.github/workflows/reusable-revised-its.yml @@ -57,6 +57,22 @@ on: AWS_SECRET_ACCESS_KEY: required: false type: string + BACKWARD_COMPATIBILITY_IT_ENABLED: + description: "Flag for backward compatibility IT" + required: true + type: string + DRUID_PREVIOUS_VERSION: + description: "Previous druid versions to run the test against." + required: true + type: string + DRUID_PREVIOUS_VERSION_DOWNLOAD_URL: + description: "URL to download the previous druid version." + required: true + type: string + DRUID_PREVIOUS_IT_IMAGE_NAME: + description: "Druid previous version image name." + required: true + type: string env: MYSQL_DRIVER_CLASSNAME: ${{ inputs.mysql_driver }} # Used by tests to connect to metadata store directly. @@ -68,6 +84,10 @@ env: SEGMENT_DOWNLOAD_TIMEOUT_MINS: 5 DOCKER_CLIENT_TIMEOUT: 120 COMPOSE_HTTP_TIMEOUT: 120 + BACKWARD_COMPATIBILITY_IT_ENABLED: ${{ inputs.BACKWARD_COMPATIBILITY_IT_ENABLED }} + DRUID_PREVIOUS_VERSION: ${{ inputs.DRUID_PREVIOUS_VERSION }} + DRUID_PREVIOUS_VERSION_DOWNLOAD_URL: ${{ inputs.DRUID_PREVIOUS_VERSION_DOWNLOAD_URL }} + DRUID_PREVIOUS_IT_IMAGE_NAME: ${{ inputs.DRUID_PREVIOUS_IT_IMAGE_NAME }} jobs: test: # GitHub job that runs a given revised/new IT against retrieved cached druid docker image @@ -122,6 +142,15 @@ jobs: echo $DRUID_IT_IMAGE_NAME docker save "$DRUID_IT_IMAGE_NAME" | gzip > druid-container-jdk${{ inputs.build_jdk }}.tar.gz + - name: Save old version docker image + if: (${{ inputs.BACKWARD_COMPATIBILITY_IT_ENABLED == 'true' }} && (steps.docker-restore.outputs.cache-hit != 'true' || steps.maven-restore.outputs.cache-hit != 'true') + env: + docker-restore: ${{ toJson(steps.docker-restore.outputs) }} + run: | + docker tag $DRUID_PREVIOUS_IT_IMAGE_NAME $DRUID_PREVIOUS_IT_IMAGE_NAME-jdk${{ inputs.build_jdk }}-version${{ inputs.DRUID_PREVIOUS_VERSION }} + echo $DRUID_PREVIOUS_IT_IMAGE_NAME + docker save "$DRUID_PREVIOUS_IT_IMAGE_NAME" | gzip > druid-container-jdk${{ inputs.build_jdk }}-version${{ inputs.DRUID_PREVIOUS_VERSION }}.tar.gz + - name: Stop and remove docker containers run: | echo "Force stopping all containers and pruning" @@ -133,6 +162,11 @@ jobs: docker load --input druid-container-jdk${{ inputs.build_jdk }}.tar.gz docker images + - name: Load previous version docker image + if: ${{ inputs.BACKWARD_COMPATIBILITY_IT_ENABLED == 'true' }} + run: | + docker load --input druid-container-jdk8-version${{ inputs.DRUID_PREVIOUS_VERSION }}.tar.gz + - name: Run IT id: run-it run: ${{ inputs.script }} diff --git a/.github/workflows/revised-its.yml b/.github/workflows/revised-its.yml index 069562bf7bd3..62c2f2b122fd 100644 --- a/.github/workflows/revised-its.yml +++ b/.github/workflows/revised-its.yml @@ -18,10 +18,28 @@ name: "Revised ITs workflow" on: workflow_call: + inputs: + BACKWARD_COMPATIBILITY_IT_ENABLED: + description: "Flag for backward compatibility IT" + required: true + type: string + DRUID_PREVIOUS_VERSION: + description: "Previous druid versions to run the test against." + required: true + type: string + DRUID_PREVIOUS_VERSION_DOWNLOAD_URL: + description: "URL to download the previous druid version." + required: true + type: string + DRUID_PREVIOUS_IT_IMAGE_NAME: + description: "Druid previous version image name." + required: true + type: string workflow_dispatch: jobs: changes: + if: false runs-on: ubuntu-latest # Required permissions permissions: @@ -44,13 +62,14 @@ jobs: - '!extension*/**' it: - needs: changes + #needs: changes strategy: fail-fast: false matrix: #jdk: [8, 11, 17] jdk: [8] - it: [HighAvailability, MultiStageQuery, Catalog, BatchIndex, MultiStageQueryWithMM, InputSource, InputFormat, Security, Query] + #it: [HighAvailability, MultiStageQuery, Catalog, BatchIndex, MultiStageQueryWithMM, InputSource, InputFormat, Security, Query, BackwardCompatibility] + it: [BackwardCompatibility] #indexer: [indexer, middleManager] indexer: [middleManager] uses: ./.github/workflows/reusable-revised-its.yml @@ -62,11 +81,15 @@ jobs: script: ./it.sh github ${{ matrix.it }} it: ${{ matrix.it }} mysql_driver: com.mysql.jdbc.Driver + BACKWARD_COMPATIBILITY_IT_ENABLED: ${{ inputs.BACKWARD_COMPATIBILITY_IT_ENABLED }} + DRUID_PREVIOUS_VERSION: ${{ inputs.DRUID_PREVIOUS_VERSION }} + DRUID_PREVIOUS_VERSION_DOWNLOAD_URL: ${{ inputs.DRUID_PREVIOUS_VERSION_DOWNLOAD_URL }} + DRUID_PREVIOUS_IT_IMAGE_NAME: ${{ inputs.DRUID_PREVIOUS_IT_IMAGE_NAME }} s3-deep-storage-minio: needs: changes uses: ./.github/workflows/reusable-revised-its.yml - if: ${{ needs.changes.outputs.core == 'true' || needs.changes.outputs.common-extensions == 'true' }} + if: false #${{ needs.changes.outputs.core == 'true' || needs.changes.outputs.common-extensions == 'true' }} with: build_jdk: 8 runtime_jdk: 11 diff --git a/.github/workflows/unit-and-integration-tests-unified.yml b/.github/workflows/unit-and-integration-tests-unified.yml index 9651a56b8cb7..1abc4d9057db 100644 --- a/.github/workflows/unit-and-integration-tests-unified.yml +++ b/.github/workflows/unit-and-integration-tests-unified.yml @@ -47,6 +47,10 @@ concurrency: env: MYSQL_DRIVER_CLASSNAME: com.mysql.jdbc.Driver # Used to set druid config in docker image for revised ITs SEGMENT_DOWNLOAD_TIMEOUT_MINS: 5 + BACKWARD_COMPATIBILITY_IT_ENABLED: false + DRUID_PREVIOUS_VERSION: 30.0.0 + DRUID_PREVIOUS_VERSION_DOWNLOAD_URL: https://dlcdn.apache.org/druid/30.0.0/apache-druid-30.0.0-bin.tar.gz + DRUID_PREVIOUS_IT_IMAGE_NAME: org.apache.druid.integration-tests/test:30.0.0 jobs: build: @@ -111,6 +115,13 @@ jobs: echo $DRUID_IT_IMAGE_NAME docker save "$DRUID_IT_IMAGE_NAME" | gzip > druid-container-jdk${{ matrix.jdk }}.tar.gz + - name: Save old version docker image + if: (${{ env.BACKWARD_COMPATIBILITY_IT_ENABLED == 'true' }} + run: | + docker tag $DRUID_PREVIOUS_IT_IMAGE_NAME $DRUID_PREVIOUS_IT_IMAGE_NAME-jdk${{ inputs.build_jdk }}-version${{ inputs.DRUID_PREVIOUS_VERSION }} + echo $DRUID_PREVIOUS_IT_IMAGE_NAME + docker save "$DRUID_PREVIOUS_IT_IMAGE_NAME" | gzip > druid-container-jdk${{ inputs.build_jdk }}-version${{ inputs.DRUID_PREVIOUS_VERSION }}.tar.gz + unit-tests-phase2: strategy: fail-fast: false @@ -119,12 +130,13 @@ jobs: name: "unit tests (jdk${{ matrix.jdk }}, sql-compat=true)" uses: ./.github/workflows/unit-tests.yml needs: unit-tests - if: ${{ always() && (needs.unit-tests.result == 'success' || needs.unit-tests.outputs.continue_tests) }} + if: false #${{ always() && (needs.unit-tests.result == 'success' || needs.unit-tests.outputs.continue_tests) }} with: jdk: ${{ matrix.jdk }} sql_compatibility: true unit-tests: + if: false strategy: fail-fast: false matrix: @@ -138,10 +150,15 @@ jobs: standard-its: needs: unit-tests - if: ${{ always() && (needs.unit-tests.result == 'success' || needs.unit-tests.outputs.continue_tests) }} + if: false #${{ always() && (needs.unit-tests.result == 'success' || needs.unit-tests.outputs.continue_tests) }} uses: ./.github/workflows/standard-its.yml revised-its: - needs: unit-tests - if: ${{ always() && (needs.unit-tests.result == 'success' || needs.unit-tests.outputs.continue_tests) }} + #needs: unit-tests + if: ${{ always() }} #&& (needs.unit-tests.result == 'success' || needs.unit-tests.outputs.continue_tests) }} uses: ./.github/workflows/revised-its.yml + with: + BACKWARD_COMPATIBILITY_IT_ENABLED: ${{ env.BACKWARD_COMPATIBILITY_IT_ENABLED }} + DRUID_PREVIOUS_VERSION: ${{ env.DRUID_PREVIOUS_VERSION }} + DRUID_PREVIOUS_VERSION_DOWNLOAD_URL: ${{ env.DRUID_PREVIOUS_VERSION_DOWNLOAD_URL }} + DRUID_PREVIOUS_IT_IMAGE_NAME: ${{ env.DRUID_PREVIOUS_IT_IMAGE_NAME }} diff --git a/docs/ingestion/input-sources.md b/docs/ingestion/input-sources.md index fb8e1f98c91f..6d8434d5d537 100644 --- a/docs/ingestion/input-sources.md +++ b/docs/ingestion/input-sources.md @@ -87,7 +87,7 @@ Sample specs: "type": "s3", "objectGlob": "**.json", "objects": [ - { "bucket": "foo", "path": "bar/file1.json"}, + { "bucket": "foo", "path": "bar/file1.yml.json"}, { "bucket": "bar", "path": "foo/file2.json"} ] }, @@ -275,7 +275,7 @@ Sample specs: "type": "google", "objectGlob": "**.json", "objects": [ - { "bucket": "foo", "path": "bar/file1.json"}, + { "bucket": "foo", "path": "bar/file1.yml.json"}, { "bucket": "bar", "path": "foo/file2.json"} ] }, @@ -362,7 +362,7 @@ Sample specs: "type": "azureStorage", "objectGlob": "**.json", "objects": [ - { "bucket": "storageAccount", "path": "container/prefix1/file1.json"}, + { "bucket": "storageAccount", "path": "container/prefix1/file1.yml.json"}, { "bucket": "storageAccount", "path": "container/prefix2/file2.json"} ], "properties": { @@ -459,7 +459,7 @@ Sample specs: "type": "azure", "objectGlob": "**.json", "objects": [ - { "bucket": "container", "path": "prefix1/file1.json"}, + { "bucket": "container", "path": "prefix1/file1.yml.json"}, { "bucket": "container", "path": "prefix2/file2.json"} ] }, diff --git a/integration-tests-ex/cases/cluster/BackwardCompatibility/docker-compose.yaml b/integration-tests-ex/cases/cluster/BackwardCompatibility/docker-compose.yaml index 0b215462e082..fe71ad2b25b2 100644 --- a/integration-tests-ex/cases/cluster/BackwardCompatibility/docker-compose.yaml +++ b/integration-tests-ex/cases/cluster/BackwardCompatibility/docker-compose.yaml @@ -35,7 +35,7 @@ services: extends: file: ../Common/druid.yaml service: coordinator - image: ${DRUID_PREV_IT_IMAGE_NAME} + image: ${DRUID_PREVIOUS_IT_IMAGE_NAME} container_name: coordinator environment: - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} @@ -52,6 +52,7 @@ services: extends: file: ../Common/druid.yaml service: overlord + image: ${DRUID_PREVIOUS_IT_IMAGE_NAME} container_name: overlord environment: - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} 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 index 542b89558fc2..8f8dc83d40f8 100644 --- 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 @@ -29,6 +29,7 @@ import org.apache.druid.testing.utils.ITRetryUtil; import org.apache.druid.testsEx.categories.BackwardCompatibility; import org.apache.druid.testsEx.categories.BatchIndex; +import org.apache.druid.testsEx.categories.InputSource; import org.apache.druid.testsEx.config.DruidTestRunner; import org.joda.time.Interval; import org.junit.Assert; diff --git a/integration-tests-ex/image/build-image.sh b/integration-tests-ex/image/build-image.sh index 51a8102d1cc3..2a6379631857 100755 --- a/integration-tests-ex/image/build-image.sh +++ b/integration-tests-ex/image/build-image.sh @@ -44,7 +44,8 @@ export MARIADB_VERSION=$MARIADB_VERSION export HADOOP_VERSION=$HADOOP_VERSION export DRUID_IT_IMAGE_NAME=$DRUID_IT_IMAGE_NAME export DRUID_PREV_VERSION=$DRUID_PREV_VERSION -export DRUID_PREV_IT_IMAGE_NAME=$DRUID_PREV_IT_IMAGE_NAME +export DRUID_PREVIOUS_VERSION_DOWNLOAD_URL=$DRUID_PREVIOUS_VERSION_DOWNLOAD_URL +export DRUID_PREVIOUS_IT_IMAGE_NAME=$DRUID_PREVIOUS_IT_IMAGE_NAME EOF exec bash $SCRIPT_DIR/docker-build.sh diff --git a/integration-tests-ex/image/docker-build.sh b/integration-tests-ex/image/docker-build.sh index dfbc0fe84531..de806621a7da 100755 --- a/integration-tests-ex/image/docker-build.sh +++ b/integration-tests-ex/image/docker-build.sh @@ -48,9 +48,6 @@ if [[ ! -f $DISTRIB_FILE || $SOURCE_FILE -nt $DISTRIB_FILE ]]; then cp $SOURCE_FILE . fi -# Download the previous druid tar -curl -L https://dlcdn.apache.org/druid/30.0.0/apache-druid-30.0.0-bin.tar.gz --output apache-druid-30.0.0-bin.tar.gz - docker build -t $DRUID_IT_IMAGE_NAME \ --build-arg DRUID_VERSION=$DRUID_VERSION \ --build-arg MYSQL_VERSION=$MYSQL_VERSION \ @@ -61,7 +58,16 @@ docker build -t $DRUID_IT_IMAGE_NAME \ --build-arg DRUID_TESTING_TOOLS_VERSION=$DRUID_VERSION \ . -docker build -t $DRUID_PREV_IT_IMAGE_NAME \ +if [ $BACKWARD_COMPATIBILITY_IT_ENABLED != "true" ]; then + exit 1 +fi + +echo "Building previous docker image" + +# Download the previous druid tar +curl -L $DRUID_PREVIOUS_VERSION_DOWNLOAD_URL --output apache-druid-$DRUID_PREV_VERSION-bin.tar.gz + +docker build -t $DRUID_PREVIOUS_IT_IMAGE_NAME \ --build-arg DRUID_VERSION=$DRUID_PREV_VERSION \ --build-arg MYSQL_VERSION=$MYSQL_VERSION \ --build-arg MARIADB_VERSION=$MARIADB_VERSION \ diff --git a/integration-tests-ex/image/pom.xml b/integration-tests-ex/image/pom.xml index f359c14dc3a2..7cac87dfd506 100644 --- a/integration-tests-ex/image/pom.xml +++ b/integration-tests-ex/image/pom.xml @@ -56,7 +56,9 @@ Reference: https://dzone.com/articles/build-docker-image-from-maven override this default name. --> ${project.groupId}/test:${project.version} 30.0.0 + https://dlcdn.apache.org/druid/30.0.0/apache-druid-30.0.0-bin.tar.gz ${project.groupId}/test:${druid.it.prev.version} + true 5.5.1 2.7.3 5.7-debian @@ -212,7 +214,9 @@ Reference: https://dzone.com/articles/build-docker-image-from-maven ${project.version} ${druid.it.image-name} ${druid.it.prev.version} - ${druid.it.prev.image-name} + ${druid.it.prev.version.download.url} + ${druid.it.prev.image-name} + ${druid.it.backward-compatibility.enabled} ${project.build.directory} From d04ffbcd6191fac8e1c607cb28fcbcd049e52c92 Mon Sep 17 00:00:00 2001 From: rishabh singh Date: Tue, 16 Jul 2024 22:41:51 +0530 Subject: [PATCH 05/31] minor change --- .github/workflows/reusable-revised-its.yml | 4 ++-- .github/workflows/unit-and-integration-tests-unified.yml | 6 +++--- integration-tests-ex/cases/pom.xml | 2 +- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/.github/workflows/reusable-revised-its.yml b/.github/workflows/reusable-revised-its.yml index 0353ece0b7f8..1d793eed635c 100644 --- a/.github/workflows/reusable-revised-its.yml +++ b/.github/workflows/reusable-revised-its.yml @@ -143,7 +143,7 @@ jobs: docker save "$DRUID_IT_IMAGE_NAME" | gzip > druid-container-jdk${{ inputs.build_jdk }}.tar.gz - name: Save old version docker image - if: (${{ inputs.BACKWARD_COMPATIBILITY_IT_ENABLED == 'true' }} && (steps.docker-restore.outputs.cache-hit != 'true' || steps.maven-restore.outputs.cache-hit != 'true') + if: ${{ inputs.BACKWARD_COMPATIBILITY_IT_ENABLED == "true" }} && (steps.docker-restore.outputs.cache-hit != 'true' || steps.maven-restore.outputs.cache-hit != 'true') env: docker-restore: ${{ toJson(steps.docker-restore.outputs) }} run: | @@ -163,7 +163,7 @@ jobs: docker images - name: Load previous version docker image - if: ${{ inputs.BACKWARD_COMPATIBILITY_IT_ENABLED == 'true' }} + if: ${{ inputs.BACKWARD_COMPATIBILITY_IT_ENABLED == "true" }} run: | docker load --input druid-container-jdk8-version${{ inputs.DRUID_PREVIOUS_VERSION }}.tar.gz diff --git a/.github/workflows/unit-and-integration-tests-unified.yml b/.github/workflows/unit-and-integration-tests-unified.yml index 1abc4d9057db..cf8255c83718 100644 --- a/.github/workflows/unit-and-integration-tests-unified.yml +++ b/.github/workflows/unit-and-integration-tests-unified.yml @@ -116,7 +116,7 @@ jobs: docker save "$DRUID_IT_IMAGE_NAME" | gzip > druid-container-jdk${{ matrix.jdk }}.tar.gz - name: Save old version docker image - if: (${{ env.BACKWARD_COMPATIBILITY_IT_ENABLED == 'true' }} + if: ${{ env.BACKWARD_COMPATIBILITY_IT_ENABLED == "true" }} run: | docker tag $DRUID_PREVIOUS_IT_IMAGE_NAME $DRUID_PREVIOUS_IT_IMAGE_NAME-jdk${{ inputs.build_jdk }}-version${{ inputs.DRUID_PREVIOUS_VERSION }} echo $DRUID_PREVIOUS_IT_IMAGE_NAME @@ -154,8 +154,8 @@ jobs: uses: ./.github/workflows/standard-its.yml revised-its: - #needs: unit-tests - if: ${{ always() }} #&& (needs.unit-tests.result == 'success' || needs.unit-tests.outputs.continue_tests) }} + needs: build + if: true #${{ always() && (needs.unit-tests.result == 'success' || needs.unit-tests.outputs.continue_tests) }} uses: ./.github/workflows/revised-its.yml with: BACKWARD_COMPATIBILITY_IT_ENABLED: ${{ env.BACKWARD_COMPATIBILITY_IT_ENABLED }} diff --git a/integration-tests-ex/cases/pom.xml b/integration-tests-ex/cases/pom.xml index 26d170ed9562..a07dad2d14fe 100644 --- a/integration-tests-ex/cases/pom.xml +++ b/integration-tests-ex/cases/pom.xml @@ -498,7 +498,7 @@ False - + org.apache.druid.testsEx.categories.${it.category} From 814c7ed3a6ee0b63779fea4e092bd73cf805b423 Mon Sep 17 00:00:00 2001 From: rishabh singh Date: Tue, 16 Jul 2024 22:42:36 +0530 Subject: [PATCH 06/31] test changes --- .github/workflows/unit-and-integration-tests-unified.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/unit-and-integration-tests-unified.yml b/.github/workflows/unit-and-integration-tests-unified.yml index cf8255c83718..c81adfd91e2b 100644 --- a/.github/workflows/unit-and-integration-tests-unified.yml +++ b/.github/workflows/unit-and-integration-tests-unified.yml @@ -58,7 +58,7 @@ jobs: strategy: fail-fast: false matrix: - jdk: [ '8', '11', '17', '21' ] + jdk: [ '8'] runs-on: ubuntu-latest steps: - name: Checkout branch From fb9ae06a370f770aa3a1b85c9e9e1b9a81c56f24 Mon Sep 17 00:00:00 2001 From: rishabh singh Date: Tue, 16 Jul 2024 23:03:56 +0530 Subject: [PATCH 07/31] temporarily disable static-checks and codeql --- .github/workflows/codeql.yml | 32 ++++++++++++++--------------- .github/workflows/static-checks.yml | 22 ++++++++++---------- 2 files changed, 27 insertions(+), 27 deletions(-) diff --git a/.github/workflows/codeql.yml b/.github/workflows/codeql.yml index 7cfeb63cbff0..899f9e744d49 100644 --- a/.github/workflows/codeql.yml +++ b/.github/workflows/codeql.yml @@ -1,21 +1,21 @@ name: "CodeQL" -on: - push: - paths-ignore: - - '**/*.md' - - 'dev/**' - - 'docs/**' - branches: [ 'master', '0.6.x', '0.7.x', '0.7.1.x', '0.7.2.x', '0.8.0', '0.8.1', '0.8.2', '0.8.3', '0.9.0', '0.9.1' ] - pull_request: - paths-ignore: - - '**/*.md' - - 'dev/**' - - 'docs/**' - # The branches below must be a subset of the branches above - branches: [ 'master' ] - schedule: - - cron: '18 15 * * 4' +#on: +# push: +# paths-ignore: +# - '**/*.md' +# - 'dev/**' +# - 'docs/**' +# branches: [ 'master', '0.6.x', '0.7.x', '0.7.1.x', '0.7.2.x', '0.8.0', '0.8.1', '0.8.2', '0.8.3', '0.9.0', '0.9.1' ] +# pull_request: +# paths-ignore: +# - '**/*.md' +# - 'dev/**' +# - 'docs/**' +# # The branches below must be a subset of the branches above +# branches: [ 'master' ] +# schedule: +# - cron: '18 15 * * 4' jobs: analyze: diff --git a/.github/workflows/static-checks.yml b/.github/workflows/static-checks.yml index a374cf72ccfe..a3265e93255c 100644 --- a/.github/workflows/static-checks.yml +++ b/.github/workflows/static-checks.yml @@ -14,17 +14,17 @@ # limitations under the License. name: "Static Checks CI" -on: - push: - branches: - - master - - '[0-9]+.[0-9]+.[0-9]+' # release branches - - '[0-9]+.[0-9]+.[0-9]+-[A-Za-z0-9]+' # release branches - pull_request: - branches: - - master - - '[0-9]+.[0-9]+.[0-9]+' # release branches - - '[0-9]+.[0-9]+.[0-9]+-[A-Za-z0-9]+' # release branches +#on: +# push: +# branches: +# - master +# - '[0-9]+.[0-9]+.[0-9]+' # release branches +# - '[0-9]+.[0-9]+.[0-9]+-[A-Za-z0-9]+' # release branches +# pull_request: +# branches: +# - master +# - '[0-9]+.[0-9]+.[0-9]+' # release branches +# - '[0-9]+.[0-9]+.[0-9]+-[A-Za-z0-9]+' # release branches concurrency: group: '${{ github.workflow }} @ ${{ github.event.pull_request.head.label || github.head_ref || github.run_id }}' # group workflows only on pull_requests and not on branch commits From 82ee40a83255bafa25fc8027c6d4fe6c36492a3b Mon Sep 17 00:00:00 2001 From: rishabh singh Date: Tue, 16 Jul 2024 23:05:48 +0530 Subject: [PATCH 08/31] disable checks --- .github/workflows/cron-job-its.yml | 20 ++++++------- .github/workflows/distribution-checks.yml | 34 +++++++++++------------ 2 files changed, 27 insertions(+), 27 deletions(-) diff --git a/.github/workflows/cron-job-its.yml b/.github/workflows/cron-job-its.yml index aa3f28382884..877fb2d385b6 100644 --- a/.github/workflows/cron-job-its.yml +++ b/.github/workflows/cron-job-its.yml @@ -14,16 +14,16 @@ # limitations under the License. name: "Cron Job ITs" -on: - schedule: # Runs by default on master branch - - cron: '0 3 * * *' # Runs every day at 3:00 AM UTC - pull_request: - paths: - - 'owasp-dependency-check-suppressions.xml' - branches: - - master - - '[0-9]+.[0-9]+.[0-9]+' # release branches - - '[0-9]+.[0-9]+.[0-9]+-[A-Za-z0-9]+' # release branches +#on: +# schedule: # Runs by default on master branch +# - cron: '0 3 * * *' # Runs every day at 3:00 AM UTC +# pull_request: +# paths: +# - 'owasp-dependency-check-suppressions.xml' +# branches: +# - master +# - '[0-9]+.[0-9]+.[0-9]+' # release branches +# - '[0-9]+.[0-9]+.[0-9]+-[A-Za-z0-9]+' # release branches jobs: build: diff --git a/.github/workflows/distribution-checks.yml b/.github/workflows/distribution-checks.yml index caadb16b608e..f0e6d2ef4626 100644 --- a/.github/workflows/distribution-checks.yml +++ b/.github/workflows/distribution-checks.yml @@ -14,23 +14,23 @@ # limitations under the License. name: "Distribution Checks" -on: - push: - branches: - - master - - '[0-9]+.[0-9]+.[0-9]+' # release branches - - '[0-9]+.[0-9]+.[0-9]+-[A-Za-z0-9]+' # release branches - paths: - - 'distribution/**' - - '**/pom.xml' - pull_request: - branches: - - master - - '[0-9]+.[0-9]+.[0-9]+' # release branches - - '[0-9]+.[0-9]+.[0-9]+-[A-Za-z0-9]+' # release branches - paths: - - 'distribution/**' - - '**/pom.xml' +#on: +# push: +# branches: +# - master +# - '[0-9]+.[0-9]+.[0-9]+' # release branches +# - '[0-9]+.[0-9]+.[0-9]+-[A-Za-z0-9]+' # release branches +# paths: +# - 'distribution/**' +# - '**/pom.xml' +# pull_request: +# branches: +# - master +# - '[0-9]+.[0-9]+.[0-9]+' # release branches +# - '[0-9]+.[0-9]+.[0-9]+-[A-Za-z0-9]+' # release branches +# paths: +# - 'distribution/**' +# - '**/pom.xml' jobs: docker-build: From da2ce1bea3537e0bb9b46b2dfe2a89db123b3be3 Mon Sep 17 00:00:00 2001 From: rishabh singh Date: Tue, 16 Jul 2024 23:10:29 +0530 Subject: [PATCH 09/31] test --- .../java/org/apache/druid/testsEx/indexer/ITIndexerTest.java | 1 + 1 file changed, 1 insertion(+) 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 index 8f8dc83d40f8..bc2f11b1d0a1 100644 --- 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 @@ -48,6 +48,7 @@ @Category({BackwardCompatibility.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"; From df01f4e0b526277a790da2624ace590671922d8f Mon Sep 17 00:00:00 2001 From: rishabh singh Date: Wed, 17 Jul 2024 00:01:05 +0530 Subject: [PATCH 10/31] minor change --- .github/workflows/revised-its.yml | 3 +- .../unit-and-integration-tests-unified.yml | 39 ++++++++++++------- .../druid/testsEx/indexer/ITIndexerTest.java | 2 +- 3 files changed, 27 insertions(+), 17 deletions(-) diff --git a/.github/workflows/revised-its.yml b/.github/workflows/revised-its.yml index 62c2f2b122fd..173a4b102969 100644 --- a/.github/workflows/revised-its.yml +++ b/.github/workflows/revised-its.yml @@ -39,7 +39,6 @@ on: jobs: changes: - if: false runs-on: ubuntu-latest # Required permissions permissions: @@ -62,7 +61,7 @@ jobs: - '!extension*/**' it: - #needs: changes + needs: changes strategy: fail-fast: false matrix: diff --git a/.github/workflows/unit-and-integration-tests-unified.yml b/.github/workflows/unit-and-integration-tests-unified.yml index c81adfd91e2b..911f3127f6f5 100644 --- a/.github/workflows/unit-and-integration-tests-unified.yml +++ b/.github/workflows/unit-and-integration-tests-unified.yml @@ -14,27 +14,38 @@ # limitations under the License. name: "Unit & Integration tests CI" +#on: +# push: +# paths-ignore: +# - '**/*.md' +# - 'dev/**' +# - 'docs/**' +# - 'examples/**/jupyter-notebooks/**' +# - 'web-console/**' +# - 'website/**' +# branches: +# - master +# - '[0-9]+.[0-9]+.[0-9]+' # release branches +# - '[0-9]+.[0-9]+.[0-9]+-[A-Za-z0-9]+' # release branches +# pull_request: +# paths-ignore: +# - '**/*.md' +# - 'dev/**' +# - 'docs/**' +# - 'examples/**/jupyter-notebooks/**' +# - 'web-console/**' +# - 'website/**' +# branches: +# - master +# - '[0-9]+.[0-9]+.[0-9]+' # release branches +# - '[0-9]+.[0-9]+.[0-9]+-[A-Za-z0-9]+' # release branches on: push: - paths-ignore: - - '**/*.md' - - 'dev/**' - - 'docs/**' - - 'examples/**/jupyter-notebooks/**' - - 'web-console/**' - - 'website/**' branches: - master - '[0-9]+.[0-9]+.[0-9]+' # release branches - '[0-9]+.[0-9]+.[0-9]+-[A-Za-z0-9]+' # release branches pull_request: - paths-ignore: - - '**/*.md' - - 'dev/**' - - 'docs/**' - - 'examples/**/jupyter-notebooks/**' - - 'web-console/**' - - 'website/**' branches: - master - '[0-9]+.[0-9]+.[0-9]+' # release branches 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 index bc2f11b1d0a1..ce89c5f135a1 100644 --- 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 @@ -48,7 +48,7 @@ @Category({BackwardCompatibility.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"; From c2ba7198f4fff96aa475ad277b205a19eb3ba84a Mon Sep 17 00:00:00 2001 From: rishabh singh Date: Wed, 17 Jul 2024 00:05:57 +0530 Subject: [PATCH 11/31] test --- .github/workflows/unit-and-integration-tests-unified.yml | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/.github/workflows/unit-and-integration-tests-unified.yml b/.github/workflows/unit-and-integration-tests-unified.yml index 911f3127f6f5..32b4489baa56 100644 --- a/.github/workflows/unit-and-integration-tests-unified.yml +++ b/.github/workflows/unit-and-integration-tests-unified.yml @@ -164,6 +164,14 @@ jobs: if: false #${{ always() && (needs.unit-tests.result == 'success' || needs.unit-tests.outputs.continue_tests) }} uses: ./.github/workflows/standard-its.yml + dummy-step: + runs-on: ubuntu-latest + name: "Dummy step" + steps: + - name: "Echo" + run: | + echo "run dummy-step" + revised-its: needs: build if: true #${{ always() && (needs.unit-tests.result == 'success' || needs.unit-tests.outputs.continue_tests) }} From 499eee9f15848a437add4ef9276b77b49d207783 Mon Sep 17 00:00:00 2001 From: rishabh singh Date: Wed, 17 Jul 2024 00:07:09 +0530 Subject: [PATCH 12/31] test --- .../druid/segment/metadata/AbstractSegmentMetadataCache.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/server/src/main/java/org/apache/druid/segment/metadata/AbstractSegmentMetadataCache.java b/server/src/main/java/org/apache/druid/segment/metadata/AbstractSegmentMetadataCache.java index 9cb2297db828..1d8bbe38d434 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/AbstractSegmentMetadataCache.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/AbstractSegmentMetadataCache.java @@ -106,6 +106,8 @@ */ public abstract class AbstractSegmentMetadataCache { + + private static final EmittingLogger log = new EmittingLogger(AbstractSegmentMetadataCache.class); private static final int MAX_SEGMENTS_PER_QUERY = 15000; private static final long DEFAULT_NUM_ROWS = 0; From ffe363dabf56206f0f87841ab18c54843ce7dede Mon Sep 17 00:00:00 2001 From: rishabh singh Date: Wed, 17 Jul 2024 00:08:24 +0530 Subject: [PATCH 13/31] test --- .../unit-and-integration-tests-unified.yml | 19 +++++-------------- .../AbstractSegmentMetadataCache.java | 2 +- 2 files changed, 6 insertions(+), 15 deletions(-) diff --git a/.github/workflows/unit-and-integration-tests-unified.yml b/.github/workflows/unit-and-integration-tests-unified.yml index 32b4489baa56..faaed23e3bb4 100644 --- a/.github/workflows/unit-and-integration-tests-unified.yml +++ b/.github/workflows/unit-and-integration-tests-unified.yml @@ -40,20 +40,11 @@ name: "Unit & Integration tests CI" # - '[0-9]+.[0-9]+.[0-9]+' # release branches # - '[0-9]+.[0-9]+.[0-9]+-[A-Za-z0-9]+' # release branches on: - push: - branches: - - master - - '[0-9]+.[0-9]+.[0-9]+' # release branches - - '[0-9]+.[0-9]+.[0-9]+-[A-Za-z0-9]+' # release branches - pull_request: - branches: - - master - - '[0-9]+.[0-9]+.[0-9]+' # release branches - - '[0-9]+.[0-9]+.[0-9]+-[A-Za-z0-9]+' # release branches - -concurrency: - group: '${{ github.workflow }} @ ${{ github.event.pull_request.head.label || github.head_ref || github.run_id }}' # group workflows only on pull_requests and not on branch commits - cancel-in-progress: true + - pull_request_target + +#concurrency: +# group: '${{ github.workflow }} @ ${{ github.event.pull_request.head.label || github.head_ref || github.run_id }}' # group workflows only on pull_requests and not on branch commits +# cancel-in-progress: true env: MYSQL_DRIVER_CLASSNAME: com.mysql.jdbc.Driver # Used to set druid config in docker image for revised ITs diff --git a/server/src/main/java/org/apache/druid/segment/metadata/AbstractSegmentMetadataCache.java b/server/src/main/java/org/apache/druid/segment/metadata/AbstractSegmentMetadataCache.java index 1d8bbe38d434..41ce236832d2 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/AbstractSegmentMetadataCache.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/AbstractSegmentMetadataCache.java @@ -107,7 +107,7 @@ public abstract class AbstractSegmentMetadataCache { - + private static final EmittingLogger log = new EmittingLogger(AbstractSegmentMetadataCache.class); private static final int MAX_SEGMENTS_PER_QUERY = 15000; private static final long DEFAULT_NUM_ROWS = 0; From fe737151c4ecb10947477295f3c92ca88bfe9417 Mon Sep 17 00:00:00 2001 From: rishabh singh Date: Wed, 17 Jul 2024 00:11:16 +0530 Subject: [PATCH 14/31] test --- .github/workflows/static-checks.yml | 27 +++++++++++++++------------ 1 file changed, 15 insertions(+), 12 deletions(-) diff --git a/.github/workflows/static-checks.yml b/.github/workflows/static-checks.yml index a3265e93255c..d1a7cc5f7173 100644 --- a/.github/workflows/static-checks.yml +++ b/.github/workflows/static-checks.yml @@ -14,17 +14,17 @@ # limitations under the License. name: "Static Checks CI" -#on: -# push: -# branches: -# - master -# - '[0-9]+.[0-9]+.[0-9]+' # release branches -# - '[0-9]+.[0-9]+.[0-9]+-[A-Za-z0-9]+' # release branches -# pull_request: -# branches: -# - master -# - '[0-9]+.[0-9]+.[0-9]+' # release branches -# - '[0-9]+.[0-9]+.[0-9]+-[A-Za-z0-9]+' # release branches +on: + push: + branches: + - master + - '[0-9]+.[0-9]+.[0-9]+' # release branches + - '[0-9]+.[0-9]+.[0-9]+-[A-Za-z0-9]+' # release branches + pull_request: + branches: + - master + - '[0-9]+.[0-9]+.[0-9]+' # release branches + - '[0-9]+.[0-9]+.[0-9]+-[A-Za-z0-9]+' # release branches concurrency: group: '${{ github.workflow }} @ ${{ github.event.pull_request.head.label || github.head_ref || github.run_id }}' # group workflows only on pull_requests and not on branch commits @@ -41,7 +41,7 @@ jobs: strategy: fail-fast: false matrix: - java: [ '8', '11', '17', '21' ] + java: [ '8'] runs-on: ubuntu-latest steps: - name: checkout branch @@ -116,6 +116,7 @@ jobs: strategy: fail-fast: false runs-on: ubuntu-latest + if: false steps: - name: checkout branch uses: actions/checkout@v4 @@ -146,6 +147,7 @@ jobs: openrewrite: runs-on: ubuntu-latest + if: false steps: - name: checkout branch uses: actions/checkout@v4 @@ -178,6 +180,7 @@ jobs: strategy: fail-fast: false runs-on: ubuntu-latest + if: false steps: - name: checkout branch uses: actions/checkout@v4 From 1800bfbfccb1329563673610ab43871f8ab4c5b4 Mon Sep 17 00:00:00 2001 From: rishabh singh Date: Wed, 17 Jul 2024 00:14:11 +0530 Subject: [PATCH 15/31] test --- .github/workflows/unit-and-integration-tests-unified.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/unit-and-integration-tests-unified.yml b/.github/workflows/unit-and-integration-tests-unified.yml index faaed23e3bb4..12999290b96b 100644 --- a/.github/workflows/unit-and-integration-tests-unified.yml +++ b/.github/workflows/unit-and-integration-tests-unified.yml @@ -60,7 +60,7 @@ jobs: strategy: fail-fast: false matrix: - jdk: [ '8'] + jdk: [ '8' ] runs-on: ubuntu-latest steps: - name: Checkout branch From 5e8e56a8b565dd9e348caca66fb46e0acf047d0a Mon Sep 17 00:00:00 2001 From: rishabh singh Date: Wed, 17 Jul 2024 00:14:41 +0530 Subject: [PATCH 16/31] test --- .github/workflows/file1.yml | 14 ++++++++++++++ .github/workflows/file2.yml | 14 ++++++++++++++ 2 files changed, 28 insertions(+) diff --git a/.github/workflows/file1.yml b/.github/workflows/file1.yml index 16812b1ff729..dc38608a4e47 100644 --- a/.github/workflows/file1.yml +++ b/.github/workflows/file1.yml @@ -1,3 +1,17 @@ +# 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. name: "A1" diff --git a/.github/workflows/file2.yml b/.github/workflows/file2.yml index 4be4773d85d9..9a4528ea33b0 100644 --- a/.github/workflows/file2.yml +++ b/.github/workflows/file2.yml @@ -1,3 +1,17 @@ +# 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. name: "A2" on: From a8d2c683edc4cd1088c75699169684153c70e223 Mon Sep 17 00:00:00 2001 From: rishabh singh Date: Wed, 17 Jul 2024 00:22:38 +0530 Subject: [PATCH 17/31] test --- .github/workflows/static-checks.yml | 19 ++++++++++--------- .../unit-and-integration-tests-unified.yml | 6 +++--- 2 files changed, 13 insertions(+), 12 deletions(-) diff --git a/.github/workflows/static-checks.yml b/.github/workflows/static-checks.yml index d1a7cc5f7173..177bad0f5d80 100644 --- a/.github/workflows/static-checks.yml +++ b/.github/workflows/static-checks.yml @@ -54,6 +54,7 @@ jobs: cache: 'maven' - name: packaging check + if: false run: | ./.github/scripts/setup_generate_license.sh ${MVN} clean install -Prat --fail-at-end \ @@ -63,7 +64,7 @@ jobs: - name: script checks # who watches the watchers? - if: ${{ matrix.java == '8' }} + if: false #${{ matrix.java == '8' }} run: ./check_test_suite_test.py - name: (openjdk17) strict compilation @@ -80,36 +81,36 @@ jobs: ${MVN} install -q -ff -pl 'distribution' ${MAVEN_SKIP} ${MAVEN_SKIP_TESTS} - name: checkstyle - if: ${{ matrix.java == '8' }} + if: false #${{ matrix.java == '8' }} run: ${MVN} checkstyle:checkstyle --fail-at-end - name: license checks - if: ${{ matrix.java == '8' }} + if: false #${{ matrix.java == '8' }} run: ./.github/scripts/license_checks_script.sh - name: analyze dependencies - if: ${{ matrix.java == '8' }} + if: false #${{ matrix.java == '8' }} run: | ./.github/scripts/analyze_dependencies_script.sh - name: animal sniffer checks - if: ${{ matrix.java == '8' }} + if: false #${{ matrix.java == '8' }} run: ${MVN} animal-sniffer:check --fail-at-end - name: enforcer checks - if: ${{ matrix.java == '8' }} + if: false #${{ matrix.java == '8' }} run: ${MVN} enforcer:enforce --fail-at-end - name: forbidden api checks - if: ${{ matrix.java == '8' }} + if: false #${{ matrix.java == '8' }} run: ${MVN} forbiddenapis:check forbiddenapis:testCheck --fail-at-end - name: pmd checks - if: ${{ matrix.java == '8' }} + if: false #${{ matrix.java == '8' }} run: ${MVN} pmd:check --fail-at-end # TODO: consider adding pmd:cpd-check - name: spotbugs checks - if: ${{ matrix.java == '8' }} + if: false #${{ matrix.java == '8' }} run: ${MVN} spotbugs:check --fail-at-end -pl '!benchmarks' intellij-inspections: diff --git a/.github/workflows/unit-and-integration-tests-unified.yml b/.github/workflows/unit-and-integration-tests-unified.yml index 12999290b96b..46db90389825 100644 --- a/.github/workflows/unit-and-integration-tests-unified.yml +++ b/.github/workflows/unit-and-integration-tests-unified.yml @@ -42,9 +42,9 @@ name: "Unit & Integration tests CI" on: - pull_request_target -#concurrency: -# group: '${{ github.workflow }} @ ${{ github.event.pull_request.head.label || github.head_ref || github.run_id }}' # group workflows only on pull_requests and not on branch commits -# cancel-in-progress: true +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.pull_request.head.label || github.head_ref || github.run_id }}' # group workflows only on pull_requests and not on branch commits + cancel-in-progress: true env: MYSQL_DRIVER_CLASSNAME: com.mysql.jdbc.Driver # Used to set druid config in docker image for revised ITs From 75755708d4e072f8cac871178b26d67a3f800d98 Mon Sep 17 00:00:00 2001 From: rishabh singh Date: Wed, 17 Jul 2024 00:28:42 +0530 Subject: [PATCH 18/31] test --- .github/workflows/file3.yml | 103 ++++++++++++++++++++++++++---------- 1 file changed, 75 insertions(+), 28 deletions(-) diff --git a/.github/workflows/file3.yml b/.github/workflows/file3.yml index be9d86460cb7..ea0d8619d747 100644 --- a/.github/workflows/file3.yml +++ b/.github/workflows/file3.yml @@ -13,19 +13,48 @@ # See the License for the specific language governing permissions and # limitations under the License. -name: "Xyz" +name: "Unit & Integration tests CI Old" +on: + push: + paths-ignore: + - '**/*.md' + - 'dev/**' + - 'docs/**' + - 'examples/**/jupyter-notebooks/**' + - 'web-console/**' + - 'website/**' + branches: + - master + - '[0-9]+.[0-9]+.[0-9]+' # release branches + - '[0-9]+.[0-9]+.[0-9]+-[A-Za-z0-9]+' # release branches + pull_request: + paths-ignore: + - '**/*.md' + - 'dev/**' + - 'docs/**' + - 'examples/**/jupyter-notebooks/**' + - 'web-console/**' + - 'website/**' + branches: + - master + - '[0-9]+.[0-9]+.[0-9]+' # release branches + - '[0-9]+.[0-9]+.[0-9]+-[A-Za-z0-9]+' # release branches + +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.pull_request.head.label || github.head_ref || github.run_id }}' # group workflows only on pull_requests and not on branch commits + cancel-in-progress: true env: MYSQL_DRIVER_CLASSNAME: com.mysql.jdbc.Driver # Used to set druid config in docker image for revised ITs SEGMENT_DOWNLOAD_TIMEOUT_MINS: 5 - BACKWARD_COMPATIBILITY_IT_ENABLED: false - DRUID_PREVIOUS_VERSION: 30.0.0 - DRUID_PREVIOUS_VERSION_DOWNLOAD_URL: https://dlcdn.apache.org/druid/30.0.0/apache-druid-30.0.0-bin.tar.gz - DRUID_PREVIOUS_IT_IMAGE_NAME: org.apache.druid.integration-tests/test:30.0.0 jobs: - test-build: - name: "build" + build: + name: "build (jdk${{ matrix.jdk }})" + strategy: + fail-fast: false + matrix: + jdk: [ '8', '11', '17', '21' ] runs-on: ubuntu-latest steps: - name: Checkout branch @@ -36,7 +65,7 @@ jobs: - uses: actions/setup-java@v4 with: distribution: 'zulu' - java-version: 8 + java-version: ${{ matrix.jdk }} # the build step produces SNAPSHOT artifacts into the local maven repository, # we include github.sha in the cache key to make it specific to that build/jdk @@ -45,7 +74,7 @@ jobs: uses: actions/cache@v4 with: path: ~/.m2/repository - key: maven-${{ runner.os }}-8-${{ github.sha }} + key: maven-${{ runner.os }}-${{ matrix.jdk }}-${{ github.sha }} restore-keys: setup-java-Linux-maven-${{ hashFiles('**/pom.xml') }} - name: Cache targets @@ -54,15 +83,15 @@ jobs: with: path: | ./**/target - key: maven-${{ runner.os }}-8-targets-${{ github.sha }} + key: maven-${{ runner.os }}-${{ matrix.jdk }}-targets-${{ github.sha }} - name: Cache image id: docker_container uses: actions/cache@v4 with: - key: druid-container-jdk8.tar.gz-${{ github.sha }} + key: druid-container-jdk${{ matrix.jdk }}.tar.gz-${{ github.sha }} path: | - ./druid-container-jdk8.tar.gz + ./druid-container-jdk${{ matrix.jdk }}.tar.gz ./integration-tests-ex/image/target/env.sh - name: Maven build @@ -74,27 +103,45 @@ jobs: run: | ./it.sh image source ./integration-tests-ex/image/target/env.sh - docker tag $DRUID_IT_IMAGE_NAME $DRUID_IT_IMAGE_NAME-jdk8 + docker tag $DRUID_IT_IMAGE_NAME $DRUID_IT_IMAGE_NAME-jdk${{ matrix.jdk }} - name: Save docker container to archive run: | source ./integration-tests-ex/image/target/env.sh echo $DRUID_IT_IMAGE_NAME - docker save "$DRUID_IT_IMAGE_NAME" | gzip > druid-container-jdk8.tar.gz + docker save "$DRUID_IT_IMAGE_NAME" | gzip > druid-container-jdk${{ matrix.jdk }}.tar.gz - - name: Save old version docker image - if: (${{ env.BACKWARD_COMPATIBILITY_IT_ENABLED == 'true' }} - run: | - docker tag $DRUID_PREVIOUS_IT_IMAGE_NAME $DRUID_PREVIOUS_IT_IMAGE_NAME-jdk8-version${{ inputs.DRUID_PREVIOUS_VERSION }} - echo $DRUID_PREVIOUS_IT_IMAGE_NAME - docker save "$DRUID_PREVIOUS_IT_IMAGE_NAME" | gzip > druid-container-jdk8-version${{ inputs.DRUID_PREVIOUS_VERSION }}.tar.gz + unit-tests-phase2: + strategy: + fail-fast: false + matrix: + jdk: [ 11, 17, 21 ] + name: "unit tests (jdk${{ matrix.jdk }}, sql-compat=true)" + uses: ./.github/workflows/unit-tests.yml + needs: unit-tests + if: ${{ always() && (needs.unit-tests.result == 'success' || needs.unit-tests.outputs.continue_tests) }} + with: + jdk: ${{ matrix.jdk }} + sql_compatibility: true - test-revised-its: - needs: test-build - if: ${{ always() }} - uses: ./.github/workflows/revised-its.yml + unit-tests: + strategy: + fail-fast: false + matrix: + sql_compatibility: [ false, true ] + name: "unit tests (jdk8, sql-compat=${{ matrix.sql_compatibility }})" + uses: ./.github/workflows/unit-tests.yml + needs: build with: - BACKWARD_COMPATIBILITY_IT_ENABLED: ${{ env.BACKWARD_COMPATIBILITY_IT_ENABLED }} - DRUID_PREVIOUS_VERSION: ${{ env.DRUID_PREVIOUS_VERSION }} - DRUID_PREVIOUS_VERSION_DOWNLOAD_URL: ${{ env.DRUID_PREVIOUS_VERSION_DOWNLOAD_URL }} - DRUID_PREVIOUS_IT_IMAGE_NAME: ${{ env.DRUID_PREVIOUS_IT_IMAGE_NAME }} + jdk: 8 + sql_compatibility: ${{ matrix.sql_compatibility }} + + standard-its: + needs: unit-tests + if: ${{ always() && (needs.unit-tests.result == 'success' || needs.unit-tests.outputs.continue_tests) }} + uses: ./.github/workflows/standard-its.yml + + revised-its: + needs: unit-tests + if: ${{ always() && (needs.unit-tests.result == 'success' || needs.unit-tests.outputs.continue_tests) }} + uses: ./.github/workflows/revised-its.yml \ No newline at end of file From ce9d240005286ca4ed25f54b0586456126f01f18 Mon Sep 17 00:00:00 2001 From: rishabh singh Date: Wed, 17 Jul 2024 00:41:44 +0530 Subject: [PATCH 19/31] test --- .github/workflows/file1.yml | 30 --- .github/workflows/file2.yml | 34 --- .github/workflows/file3.yml | 147 ------------ .../unit-and-integration-tests-unified.yml | 221 +++++++++--------- 4 files changed, 111 insertions(+), 321 deletions(-) delete mode 100644 .github/workflows/file1.yml delete mode 100644 .github/workflows/file2.yml delete mode 100644 .github/workflows/file3.yml diff --git a/.github/workflows/file1.yml b/.github/workflows/file1.yml deleted file mode 100644 index dc38608a4e47..000000000000 --- a/.github/workflows/file1.yml +++ /dev/null @@ -1,30 +0,0 @@ -# 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. - -name: "A1" - -env: - MYSQL_DRIVER_CLASSNAME: com.mysql.jdbc.Driver # Used to set druid config in docker image for revised ITs - SEGMENT_DOWNLOAD_TIMEOUT_MINS: 5 - PREVIOUS_DRUID_VERSIONS: '["30.0.0", "29.0.0"]' - TEST_VAR: 2 - - -jobs: - mop: - if: true - uses: ./.github/workflows/file2.yml - with: - previous_versions: ${{ env.PREVIOUS_DRUID_VERSIONS }} \ No newline at end of file diff --git a/.github/workflows/file2.yml b/.github/workflows/file2.yml deleted file mode 100644 index 9a4528ea33b0..000000000000 --- a/.github/workflows/file2.yml +++ /dev/null @@ -1,34 +0,0 @@ -# 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. - -name: "A2" -on: - workflow_call: - inputs: - previous_versions: - description: "Previous druid versions to run the test against." - required: true - type: string - workflow_dispatch: - -jobs: - echo_array_var_json: - runs-on: ubuntu-latest # You can choose a different runner if needed - steps: - - name: Echo environment variable (array as JSON) - run: | - echo $TEST_VAR - echo "printing var" - echo "${{inputs.previous_versions}}" \ No newline at end of file diff --git a/.github/workflows/file3.yml b/.github/workflows/file3.yml deleted file mode 100644 index ea0d8619d747..000000000000 --- a/.github/workflows/file3.yml +++ /dev/null @@ -1,147 +0,0 @@ -# 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. - -name: "Unit & Integration tests CI Old" -on: - push: - paths-ignore: - - '**/*.md' - - 'dev/**' - - 'docs/**' - - 'examples/**/jupyter-notebooks/**' - - 'web-console/**' - - 'website/**' - branches: - - master - - '[0-9]+.[0-9]+.[0-9]+' # release branches - - '[0-9]+.[0-9]+.[0-9]+-[A-Za-z0-9]+' # release branches - pull_request: - paths-ignore: - - '**/*.md' - - 'dev/**' - - 'docs/**' - - 'examples/**/jupyter-notebooks/**' - - 'web-console/**' - - 'website/**' - branches: - - master - - '[0-9]+.[0-9]+.[0-9]+' # release branches - - '[0-9]+.[0-9]+.[0-9]+-[A-Za-z0-9]+' # release branches - -concurrency: - group: '${{ github.workflow }} @ ${{ github.event.pull_request.head.label || github.head_ref || github.run_id }}' # group workflows only on pull_requests and not on branch commits - cancel-in-progress: true - -env: - MYSQL_DRIVER_CLASSNAME: com.mysql.jdbc.Driver # Used to set druid config in docker image for revised ITs - SEGMENT_DOWNLOAD_TIMEOUT_MINS: 5 - -jobs: - build: - name: "build (jdk${{ matrix.jdk }})" - strategy: - fail-fast: false - matrix: - jdk: [ '8', '11', '17', '21' ] - runs-on: ubuntu-latest - steps: - - name: Checkout branch - uses: actions/checkout@v4 - - # skip the "cache: maven" step from setup-java. We explicitly use a - # different cache key since we cannot reuse it across commits. - - uses: actions/setup-java@v4 - with: - distribution: 'zulu' - java-version: ${{ matrix.jdk }} - - # the build step produces SNAPSHOT artifacts into the local maven repository, - # we include github.sha in the cache key to make it specific to that build/jdk - - name: Cache Maven m2 repository - id: maven - uses: actions/cache@v4 - with: - path: ~/.m2/repository - key: maven-${{ runner.os }}-${{ matrix.jdk }}-${{ github.sha }} - restore-keys: setup-java-Linux-maven-${{ hashFiles('**/pom.xml') }} - - - name: Cache targets - id: target - uses: actions/cache@v4 - with: - path: | - ./**/target - key: maven-${{ runner.os }}-${{ matrix.jdk }}-targets-${{ github.sha }} - - - name: Cache image - id: docker_container - uses: actions/cache@v4 - with: - key: druid-container-jdk${{ matrix.jdk }}.tar.gz-${{ github.sha }} - path: | - ./druid-container-jdk${{ matrix.jdk }}.tar.gz - ./integration-tests-ex/image/target/env.sh - - - name: Maven build - id: maven_build - run: | - ./it.sh ci - - - name: Container build - run: | - ./it.sh image - source ./integration-tests-ex/image/target/env.sh - docker tag $DRUID_IT_IMAGE_NAME $DRUID_IT_IMAGE_NAME-jdk${{ matrix.jdk }} - - - name: Save docker container to archive - run: | - source ./integration-tests-ex/image/target/env.sh - echo $DRUID_IT_IMAGE_NAME - docker save "$DRUID_IT_IMAGE_NAME" | gzip > druid-container-jdk${{ matrix.jdk }}.tar.gz - - unit-tests-phase2: - strategy: - fail-fast: false - matrix: - jdk: [ 11, 17, 21 ] - name: "unit tests (jdk${{ matrix.jdk }}, sql-compat=true)" - uses: ./.github/workflows/unit-tests.yml - needs: unit-tests - if: ${{ always() && (needs.unit-tests.result == 'success' || needs.unit-tests.outputs.continue_tests) }} - with: - jdk: ${{ matrix.jdk }} - sql_compatibility: true - - unit-tests: - strategy: - fail-fast: false - matrix: - sql_compatibility: [ false, true ] - name: "unit tests (jdk8, sql-compat=${{ matrix.sql_compatibility }})" - uses: ./.github/workflows/unit-tests.yml - needs: build - with: - jdk: 8 - sql_compatibility: ${{ matrix.sql_compatibility }} - - standard-its: - needs: unit-tests - if: ${{ always() && (needs.unit-tests.result == 'success' || needs.unit-tests.outputs.continue_tests) }} - uses: ./.github/workflows/standard-its.yml - - revised-its: - needs: unit-tests - if: ${{ always() && (needs.unit-tests.result == 'success' || needs.unit-tests.outputs.continue_tests) }} - uses: ./.github/workflows/revised-its.yml \ No newline at end of file diff --git a/.github/workflows/unit-and-integration-tests-unified.yml b/.github/workflows/unit-and-integration-tests-unified.yml index 46db90389825..b94824c032cc 100644 --- a/.github/workflows/unit-and-integration-tests-unified.yml +++ b/.github/workflows/unit-and-integration-tests-unified.yml @@ -55,106 +55,6 @@ env: DRUID_PREVIOUS_IT_IMAGE_NAME: org.apache.druid.integration-tests/test:30.0.0 jobs: - build: - name: "build (jdk${{ matrix.jdk }})" - strategy: - fail-fast: false - matrix: - jdk: [ '8' ] - runs-on: ubuntu-latest - steps: - - name: Checkout branch - uses: actions/checkout@v4 - - # skip the "cache: maven" step from setup-java. We explicitly use a - # different cache key since we cannot reuse it across commits. - - uses: actions/setup-java@v4 - with: - distribution: 'zulu' - java-version: ${{ matrix.jdk }} - - # the build step produces SNAPSHOT artifacts into the local maven repository, - # we include github.sha in the cache key to make it specific to that build/jdk - - name: Cache Maven m2 repository - id: maven - uses: actions/cache@v4 - with: - path: ~/.m2/repository - key: maven-${{ runner.os }}-${{ matrix.jdk }}-${{ github.sha }} - restore-keys: setup-java-Linux-maven-${{ hashFiles('**/pom.xml') }} - - - name: Cache targets - id: target - uses: actions/cache@v4 - with: - path: | - ./**/target - key: maven-${{ runner.os }}-${{ matrix.jdk }}-targets-${{ github.sha }} - - - name: Cache image - id: docker_container - uses: actions/cache@v4 - with: - key: druid-container-jdk${{ matrix.jdk }}.tar.gz-${{ github.sha }} - path: | - ./druid-container-jdk${{ matrix.jdk }}.tar.gz - ./integration-tests-ex/image/target/env.sh - - - name: Maven build - id: maven_build - run: | - ./it.sh ci - - - name: Container build - run: | - ./it.sh image - source ./integration-tests-ex/image/target/env.sh - docker tag $DRUID_IT_IMAGE_NAME $DRUID_IT_IMAGE_NAME-jdk${{ matrix.jdk }} - - - name: Save docker container to archive - run: | - source ./integration-tests-ex/image/target/env.sh - echo $DRUID_IT_IMAGE_NAME - docker save "$DRUID_IT_IMAGE_NAME" | gzip > druid-container-jdk${{ matrix.jdk }}.tar.gz - - - name: Save old version docker image - if: ${{ env.BACKWARD_COMPATIBILITY_IT_ENABLED == "true" }} - run: | - docker tag $DRUID_PREVIOUS_IT_IMAGE_NAME $DRUID_PREVIOUS_IT_IMAGE_NAME-jdk${{ inputs.build_jdk }}-version${{ inputs.DRUID_PREVIOUS_VERSION }} - echo $DRUID_PREVIOUS_IT_IMAGE_NAME - docker save "$DRUID_PREVIOUS_IT_IMAGE_NAME" | gzip > druid-container-jdk${{ inputs.build_jdk }}-version${{ inputs.DRUID_PREVIOUS_VERSION }}.tar.gz - - unit-tests-phase2: - strategy: - fail-fast: false - matrix: - jdk: [ 11, 17, 21 ] - name: "unit tests (jdk${{ matrix.jdk }}, sql-compat=true)" - uses: ./.github/workflows/unit-tests.yml - needs: unit-tests - if: false #${{ always() && (needs.unit-tests.result == 'success' || needs.unit-tests.outputs.continue_tests) }} - with: - jdk: ${{ matrix.jdk }} - sql_compatibility: true - - unit-tests: - if: false - strategy: - fail-fast: false - matrix: - sql_compatibility: [ false, true ] - name: "unit tests (jdk8, sql-compat=${{ matrix.sql_compatibility }})" - uses: ./.github/workflows/unit-tests.yml - needs: build - with: - jdk: 8 - sql_compatibility: ${{ matrix.sql_compatibility }} - - standard-its: - needs: unit-tests - if: false #${{ always() && (needs.unit-tests.result == 'success' || needs.unit-tests.outputs.continue_tests) }} - uses: ./.github/workflows/standard-its.yml - dummy-step: runs-on: ubuntu-latest name: "Dummy step" @@ -162,13 +62,114 @@ jobs: - name: "Echo" run: | echo "run dummy-step" - - revised-its: - needs: build - if: true #${{ always() && (needs.unit-tests.result == 'success' || needs.unit-tests.outputs.continue_tests) }} - uses: ./.github/workflows/revised-its.yml - with: - BACKWARD_COMPATIBILITY_IT_ENABLED: ${{ env.BACKWARD_COMPATIBILITY_IT_ENABLED }} - DRUID_PREVIOUS_VERSION: ${{ env.DRUID_PREVIOUS_VERSION }} - DRUID_PREVIOUS_VERSION_DOWNLOAD_URL: ${{ env.DRUID_PREVIOUS_VERSION_DOWNLOAD_URL }} - DRUID_PREVIOUS_IT_IMAGE_NAME: ${{ env.DRUID_PREVIOUS_IT_IMAGE_NAME }} +# +# build: +# name: "build (jdk${{ matrix.jdk }})" +# strategy: +# fail-fast: false +# matrix: +# jdk: [ '8' ] +# runs-on: ubuntu-latest +# steps: +# - name: Checkout branch +# uses: actions/checkout@v4 +# +# # skip the "cache: maven" step from setup-java. We explicitly use a +# # different cache key since we cannot reuse it across commits. +# - uses: actions/setup-java@v4 +# with: +# distribution: 'zulu' +# java-version: ${{ matrix.jdk }} +# +# # the build step produces SNAPSHOT artifacts into the local maven repository, +# # we include github.sha in the cache key to make it specific to that build/jdk +# - name: Cache Maven m2 repository +# id: maven +# uses: actions/cache@v4 +# with: +# path: ~/.m2/repository +# key: maven-${{ runner.os }}-${{ matrix.jdk }}-${{ github.sha }} +# restore-keys: setup-java-Linux-maven-${{ hashFiles('**/pom.xml') }} +# +# - name: Cache targets +# id: target +# uses: actions/cache@v4 +# with: +# path: | +# ./**/target +# key: maven-${{ runner.os }}-${{ matrix.jdk }}-targets-${{ github.sha }} +# +# - name: Cache image +# id: docker_container +# uses: actions/cache@v4 +# with: +# key: druid-container-jdk${{ matrix.jdk }}.tar.gz-${{ github.sha }} +# path: | +# ./druid-container-jdk${{ matrix.jdk }}.tar.gz +# ./integration-tests-ex/image/target/env.sh +# +# - name: Maven build +# id: maven_build +# run: | +# ./it.sh ci +# +# - name: Container build +# run: | +# ./it.sh image +# source ./integration-tests-ex/image/target/env.sh +# docker tag $DRUID_IT_IMAGE_NAME $DRUID_IT_IMAGE_NAME-jdk${{ matrix.jdk }} +# +# - name: Save docker container to archive +# run: | +# source ./integration-tests-ex/image/target/env.sh +# echo $DRUID_IT_IMAGE_NAME +# docker save "$DRUID_IT_IMAGE_NAME" | gzip > druid-container-jdk${{ matrix.jdk }}.tar.gz +# +# - name: Save old version docker image +# if: ${{ env.BACKWARD_COMPATIBILITY_IT_ENABLED == "true" }} +# run: | +# docker tag $DRUID_PREVIOUS_IT_IMAGE_NAME $DRUID_PREVIOUS_IT_IMAGE_NAME-jdk${{ inputs.build_jdk }}-version${{ inputs.DRUID_PREVIOUS_VERSION }} +# echo $DRUID_PREVIOUS_IT_IMAGE_NAME +# docker save "$DRUID_PREVIOUS_IT_IMAGE_NAME" | gzip > druid-container-jdk${{ inputs.build_jdk }}-version${{ inputs.DRUID_PREVIOUS_VERSION }}.tar.gz +# +# unit-tests-phase2: +# strategy: +# fail-fast: false +# matrix: +# jdk: [ 11, 17, 21 ] +# name: "unit tests (jdk${{ matrix.jdk }}, sql-compat=true)" +# uses: ./.github/workflows/unit-tests.yml +# needs: unit-tests +# if: false #${{ always() && (needs.unit-tests.result == 'success' || needs.unit-tests.outputs.continue_tests) }} +# with: +# jdk: ${{ matrix.jdk }} +# sql_compatibility: true +# +# unit-tests: +# if: false +# strategy: +# fail-fast: false +# matrix: +# sql_compatibility: [ false, true ] +# name: "unit tests (jdk8, sql-compat=${{ matrix.sql_compatibility }})" +# uses: ./.github/workflows/unit-tests.yml +# needs: build +# with: +# jdk: 8 +# sql_compatibility: ${{ matrix.sql_compatibility }} +# +# standard-its: +# needs: unit-tests +# if: false #${{ always() && (needs.unit-tests.result == 'success' || needs.unit-tests.outputs.continue_tests) }} +# uses: ./.github/workflows/standard-its.yml +# +# +# revised-its: +# needs: build +# if: true #${{ always() && (needs.unit-tests.result == 'success' || needs.unit-tests.outputs.continue_tests) }} +# uses: ./.github/workflows/revised-its.yml +# with: +# BACKWARD_COMPATIBILITY_IT_ENABLED: ${{ env.BACKWARD_COMPATIBILITY_IT_ENABLED }} +# DRUID_PREVIOUS_VERSION: ${{ env.DRUID_PREVIOUS_VERSION }} +# DRUID_PREVIOUS_VERSION_DOWNLOAD_URL: ${{ env.DRUID_PREVIOUS_VERSION_DOWNLOAD_URL }} +# DRUID_PREVIOUS_IT_IMAGE_NAME: ${{ env.DRUID_PREVIOUS_IT_IMAGE_NAME }} From b3b20722a06bf7cacbb84eb6d11726e2460ce70d Mon Sep 17 00:00:00 2001 From: rishabh singh Date: Wed, 17 Jul 2024 00:50:09 +0530 Subject: [PATCH 20/31] test --- .github/workflows/static-checks.yml | 6 +++--- .github/workflows/unit-and-integration-tests-unified.yml | 8 ++++---- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/.github/workflows/static-checks.yml b/.github/workflows/static-checks.yml index 177bad0f5d80..c36fc178451f 100644 --- a/.github/workflows/static-checks.yml +++ b/.github/workflows/static-checks.yml @@ -26,9 +26,9 @@ on: - '[0-9]+.[0-9]+.[0-9]+' # release branches - '[0-9]+.[0-9]+.[0-9]+-[A-Za-z0-9]+' # release branches -concurrency: - group: '${{ github.workflow }} @ ${{ github.event.pull_request.head.label || github.head_ref || github.run_id }}' # group workflows only on pull_requests and not on branch commits - cancel-in-progress: true +#concurrency: +# group: '${{ github.workflow }} @ ${{ github.event.pull_request.head.label || github.head_ref || github.run_id }}' # group workflows only on pull_requests and not on branch commits +# cancel-in-progress: true env: MVN: mvn -B diff --git a/.github/workflows/unit-and-integration-tests-unified.yml b/.github/workflows/unit-and-integration-tests-unified.yml index b94824c032cc..badf2d212629 100644 --- a/.github/workflows/unit-and-integration-tests-unified.yml +++ b/.github/workflows/unit-and-integration-tests-unified.yml @@ -41,10 +41,10 @@ name: "Unit & Integration tests CI" # - '[0-9]+.[0-9]+.[0-9]+-[A-Za-z0-9]+' # release branches on: - pull_request_target - -concurrency: - group: '${{ github.workflow }} @ ${{ github.event.pull_request.head.label || github.head_ref || github.run_id }}' # group workflows only on pull_requests and not on branch commits - cancel-in-progress: true +# +#concurrency: +# group: '${{ github.workflow }} @ ${{ github.event.pull_request.head.label || github.head_ref || github.run_id }}' # group workflows only on pull_requests and not on branch commits +# cancel-in-progress: true env: MYSQL_DRIVER_CLASSNAME: com.mysql.jdbc.Driver # Used to set druid config in docker image for revised ITs From d7120e842539c60765c445425e130465d19a9e3c Mon Sep 17 00:00:00 2001 From: rishabh singh Date: Wed, 17 Jul 2024 00:53:32 +0530 Subject: [PATCH 21/31] test --- .github/workflows/unit-and-integration-tests-unified.yml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/.github/workflows/unit-and-integration-tests-unified.yml b/.github/workflows/unit-and-integration-tests-unified.yml index badf2d212629..5b37b5fc4bde 100644 --- a/.github/workflows/unit-and-integration-tests-unified.yml +++ b/.github/workflows/unit-and-integration-tests-unified.yml @@ -40,7 +40,8 @@ name: "Unit & Integration tests CI" # - '[0-9]+.[0-9]+.[0-9]+' # release branches # - '[0-9]+.[0-9]+.[0-9]+-[A-Za-z0-9]+' # release branches on: - - pull_request_target + # Trigger on push events to any branch + push: {} # Empty curly braces indicate all branches # #concurrency: # group: '${{ github.workflow }} @ ${{ github.event.pull_request.head.label || github.head_ref || github.run_id }}' # group workflows only on pull_requests and not on branch commits From 3a5e611e83bd294631b67be5fe75263d9d7904d5 Mon Sep 17 00:00:00 2001 From: rishabh singh Date: Wed, 17 Jul 2024 00:54:41 +0530 Subject: [PATCH 22/31] test --- .github/workflows/test-action.yml | 57 +++++++++++++++++++++++++++++++ 1 file changed, 57 insertions(+) create mode 100644 .github/workflows/test-action.yml diff --git a/.github/workflows/test-action.yml b/.github/workflows/test-action.yml new file mode 100644 index 000000000000..c83e9d4baf3c --- /dev/null +++ b/.github/workflows/test-action.yml @@ -0,0 +1,57 @@ +# 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. + +name: "Test action" +#on: +# push: +# paths-ignore: +# - '**/*.md' +# - 'dev/**' +# - 'docs/**' +# - 'examples/**/jupyter-notebooks/**' +# - 'web-console/**' +# - 'website/**' +# branches: +# - master +# - '[0-9]+.[0-9]+.[0-9]+' # release branches +# - '[0-9]+.[0-9]+.[0-9]+-[A-Za-z0-9]+' # release branches +# pull_request: +# paths-ignore: +# - '**/*.md' +# - 'dev/**' +# - 'docs/**' +# - 'examples/**/jupyter-notebooks/**' +# - 'web-console/**' +# - 'website/**' +# branches: +# - master +# - '[0-9]+.[0-9]+.[0-9]+' # release branches +# - '[0-9]+.[0-9]+.[0-9]+-[A-Za-z0-9]+' # release branches +on: + # Trigger on push events to any branch + push: {} # Empty curly braces indicate all branches +# +#concurrency: +# group: '${{ github.workflow }} @ ${{ github.event.pull_request.head.label || github.head_ref || github.run_id }}' # group workflows only on pull_requests and not on branch commits +# cancel-in-progress: true + +jobs: + dummy-step: + runs-on: ubuntu-latest + name: "Dummy step" + steps: + - name: "Echo" + run: | + echo "run dummy-step" From c5a145d6d67adf14f461e632a3d864b2e0c209f4 Mon Sep 17 00:00:00 2001 From: rishabh singh Date: Wed, 17 Jul 2024 01:01:41 +0530 Subject: [PATCH 23/31] test --- .github/workflows/static-checks.yml | 119 +++++++++++++ .../unit-and-integration-tests-unified.yml | 158 +++++++++--------- 2 files changed, 198 insertions(+), 79 deletions(-) diff --git a/.github/workflows/static-checks.yml b/.github/workflows/static-checks.yml index c36fc178451f..28005ae54c89 100644 --- a/.github/workflows/static-checks.yml +++ b/.github/workflows/static-checks.yml @@ -35,6 +35,13 @@ env: MAVEN_SKIP: -P skip-static-checks -Dweb.console.skip=true -Dmaven.javadoc.skip=true MAVEN_SKIP_TESTS: -P skip-tests MAVEN_OPTS: -Xmx8g + MYSQL_DRIVER_CLASSNAME: com.mysql.jdbc.Driver # Used to set druid config in docker image for revised ITs + SEGMENT_DOWNLOAD_TIMEOUT_MINS: 5 + BACKWARD_COMPATIBILITY_IT_ENABLED: false + DRUID_PREVIOUS_VERSION: 30.0.0 + DRUID_PREVIOUS_VERSION_DOWNLOAD_URL: https://dlcdn.apache.org/druid/30.0.0/apache-druid-30.0.0-bin.tar.gz + DRUID_PREVIOUS_IT_IMAGE_NAME: org.apache.druid.integration-tests/test:30.0.0 + jobs: static-checks: @@ -230,3 +237,115 @@ jobs: with: name: Druid logs web-checks path: druid-logs.tgz + + + build: + name: "build (jdk${{ matrix.jdk }})" + strategy: + fail-fast: false + matrix: + jdk: [ '8' ] + runs-on: ubuntu-latest + steps: + - name: Checkout branch + uses: actions/checkout@v4 + + # skip the "cache: maven" step from setup-java. We explicitly use a + # different cache key since we cannot reuse it across commits. + - uses: actions/setup-java@v4 + with: + distribution: 'zulu' + java-version: ${{ matrix.jdk }} + + # the build step produces SNAPSHOT artifacts into the local maven repository, + # we include github.sha in the cache key to make it specific to that build/jdk + - name: Cache Maven m2 repository + id: maven + uses: actions/cache@v4 + with: + path: ~/.m2/repository + key: maven-${{ runner.os }}-${{ matrix.jdk }}-${{ github.sha }} + restore-keys: setup-java-Linux-maven-${{ hashFiles('**/pom.xml') }} + + - name: Cache targets + id: target + uses: actions/cache@v4 + with: + path: | + ./**/target + key: maven-${{ runner.os }}-${{ matrix.jdk }}-targets-${{ github.sha }} + + - name: Cache image + id: docker_container + uses: actions/cache@v4 + with: + key: druid-container-jdk${{ matrix.jdk }}.tar.gz-${{ github.sha }} + path: | + ./druid-container-jdk${{ matrix.jdk }}.tar.gz + ./integration-tests-ex/image/target/env.sh + + - name: Maven build + id: maven_build + run: | + ./it.sh ci + + - name: Container build + run: | + ./it.sh image + source ./integration-tests-ex/image/target/env.sh + docker tag $DRUID_IT_IMAGE_NAME $DRUID_IT_IMAGE_NAME-jdk${{ matrix.jdk }} + + - name: Save docker container to archive + run: | + source ./integration-tests-ex/image/target/env.sh + echo $DRUID_IT_IMAGE_NAME + docker save "$DRUID_IT_IMAGE_NAME" | gzip > druid-container-jdk${{ matrix.jdk }}.tar.gz + + - name: Save old version docker image + if: ${{ env.BACKWARD_COMPATIBILITY_IT_ENABLED == "true" }} + run: | + docker tag $DRUID_PREVIOUS_IT_IMAGE_NAME $DRUID_PREVIOUS_IT_IMAGE_NAME-jdk${{ inputs.build_jdk }}-version${{ inputs.DRUID_PREVIOUS_VERSION }} + echo $DRUID_PREVIOUS_IT_IMAGE_NAME + docker save "$DRUID_PREVIOUS_IT_IMAGE_NAME" | gzip > druid-container-jdk${{ inputs.build_jdk }}-version${{ inputs.DRUID_PREVIOUS_VERSION }}.tar.gz + + # unit-tests-phase2: + # strategy: + # fail-fast: false + # matrix: + # jdk: [ 11, 17, 21 ] + # name: "unit tests (jdk${{ matrix.jdk }}, sql-compat=true)" + # uses: ./.github/workflows/unit-tests.yml + # needs: unit-tests + # if: false #${{ always() && (needs.unit-tests.result == 'success' || needs.unit-tests.outputs.continue_tests) }} + # with: + # jdk: ${{ matrix.jdk }} + # sql_compatibility: true + # + # unit-tests: + # if: false + # strategy: + # fail-fast: false + # matrix: + # sql_compatibility: [ false, true ] + # name: "unit tests (jdk8, sql-compat=${{ matrix.sql_compatibility }})" + # uses: ./.github/workflows/unit-tests.yml + # needs: build + # with: + # jdk: 8 + # sql_compatibility: ${{ matrix.sql_compatibility }} + # + # standard-its: + # needs: unit-tests + # if: false #${{ always() && (needs.unit-tests.result == 'success' || needs.unit-tests.outputs.continue_tests) }} + # uses: ./.github/workflows/standard-its.yml + # + # + revised-its: + needs: build + if: true #${{ always() && (needs.unit-tests.result == 'success' || needs.unit-tests.outputs.continue_tests) }} + uses: ./.github/workflows/revised-its.yml + with: + BACKWARD_COMPATIBILITY_IT_ENABLED: ${{ env.BACKWARD_COMPATIBILITY_IT_ENABLED }} + DRUID_PREVIOUS_VERSION: ${{ env.DRUID_PREVIOUS_VERSION }} + DRUID_PREVIOUS_VERSION_DOWNLOAD_URL: ${{ env.DRUID_PREVIOUS_VERSION_DOWNLOAD_URL }} + DRUID_PREVIOUS_IT_IMAGE_NAME: ${{ env.DRUID_PREVIOUS_IT_IMAGE_NAME }} diff --git a/.github/workflows/unit-and-integration-tests-unified.yml b/.github/workflows/unit-and-integration-tests-unified.yml index 5b37b5fc4bde..24a35702b902 100644 --- a/.github/workflows/unit-and-integration-tests-unified.yml +++ b/.github/workflows/unit-and-integration-tests-unified.yml @@ -63,76 +63,76 @@ jobs: - name: "Echo" run: | echo "run dummy-step" -# -# build: -# name: "build (jdk${{ matrix.jdk }})" -# strategy: -# fail-fast: false -# matrix: -# jdk: [ '8' ] -# runs-on: ubuntu-latest -# steps: -# - name: Checkout branch -# uses: actions/checkout@v4 -# -# # skip the "cache: maven" step from setup-java. We explicitly use a -# # different cache key since we cannot reuse it across commits. -# - uses: actions/setup-java@v4 -# with: -# distribution: 'zulu' -# java-version: ${{ matrix.jdk }} -# -# # the build step produces SNAPSHOT artifacts into the local maven repository, -# # we include github.sha in the cache key to make it specific to that build/jdk -# - name: Cache Maven m2 repository -# id: maven -# uses: actions/cache@v4 -# with: -# path: ~/.m2/repository -# key: maven-${{ runner.os }}-${{ matrix.jdk }}-${{ github.sha }} -# restore-keys: setup-java-Linux-maven-${{ hashFiles('**/pom.xml') }} -# -# - name: Cache targets -# id: target -# uses: actions/cache@v4 -# with: -# path: | -# ./**/target -# key: maven-${{ runner.os }}-${{ matrix.jdk }}-targets-${{ github.sha }} -# -# - name: Cache image -# id: docker_container -# uses: actions/cache@v4 -# with: -# key: druid-container-jdk${{ matrix.jdk }}.tar.gz-${{ github.sha }} -# path: | -# ./druid-container-jdk${{ matrix.jdk }}.tar.gz -# ./integration-tests-ex/image/target/env.sh -# -# - name: Maven build -# id: maven_build -# run: | -# ./it.sh ci -# -# - name: Container build -# run: | -# ./it.sh image -# source ./integration-tests-ex/image/target/env.sh -# docker tag $DRUID_IT_IMAGE_NAME $DRUID_IT_IMAGE_NAME-jdk${{ matrix.jdk }} -# -# - name: Save docker container to archive -# run: | -# source ./integration-tests-ex/image/target/env.sh -# echo $DRUID_IT_IMAGE_NAME -# docker save "$DRUID_IT_IMAGE_NAME" | gzip > druid-container-jdk${{ matrix.jdk }}.tar.gz -# -# - name: Save old version docker image -# if: ${{ env.BACKWARD_COMPATIBILITY_IT_ENABLED == "true" }} -# run: | -# docker tag $DRUID_PREVIOUS_IT_IMAGE_NAME $DRUID_PREVIOUS_IT_IMAGE_NAME-jdk${{ inputs.build_jdk }}-version${{ inputs.DRUID_PREVIOUS_VERSION }} -# echo $DRUID_PREVIOUS_IT_IMAGE_NAME -# docker save "$DRUID_PREVIOUS_IT_IMAGE_NAME" | gzip > druid-container-jdk${{ inputs.build_jdk }}-version${{ inputs.DRUID_PREVIOUS_VERSION }}.tar.gz -# + + build: + name: "build (jdk${{ matrix.jdk }})" + strategy: + fail-fast: false + matrix: + jdk: [ '8' ] + runs-on: ubuntu-latest + steps: + - name: Checkout branch + uses: actions/checkout@v4 + + # skip the "cache: maven" step from setup-java. We explicitly use a + # different cache key since we cannot reuse it across commits. + - uses: actions/setup-java@v4 + with: + distribution: 'zulu' + java-version: ${{ matrix.jdk }} + + # the build step produces SNAPSHOT artifacts into the local maven repository, + # we include github.sha in the cache key to make it specific to that build/jdk + - name: Cache Maven m2 repository + id: maven + uses: actions/cache@v4 + with: + path: ~/.m2/repository + key: maven-${{ runner.os }}-${{ matrix.jdk }}-${{ github.sha }} + restore-keys: setup-java-Linux-maven-${{ hashFiles('**/pom.xml') }} + + - name: Cache targets + id: target + uses: actions/cache@v4 + with: + path: | + ./**/target + key: maven-${{ runner.os }}-${{ matrix.jdk }}-targets-${{ github.sha }} + + - name: Cache image + id: docker_container + uses: actions/cache@v4 + with: + key: druid-container-jdk${{ matrix.jdk }}.tar.gz-${{ github.sha }} + path: | + ./druid-container-jdk${{ matrix.jdk }}.tar.gz + ./integration-tests-ex/image/target/env.sh + + - name: Maven build + id: maven_build + run: | + ./it.sh ci + + - name: Container build + run: | + ./it.sh image + source ./integration-tests-ex/image/target/env.sh + docker tag $DRUID_IT_IMAGE_NAME $DRUID_IT_IMAGE_NAME-jdk${{ matrix.jdk }} + + - name: Save docker container to archive + run: | + source ./integration-tests-ex/image/target/env.sh + echo $DRUID_IT_IMAGE_NAME + docker save "$DRUID_IT_IMAGE_NAME" | gzip > druid-container-jdk${{ matrix.jdk }}.tar.gz + + - name: Save old version docker image + if: ${{ env.BACKWARD_COMPATIBILITY_IT_ENABLED == "true" }} + run: | + docker tag $DRUID_PREVIOUS_IT_IMAGE_NAME $DRUID_PREVIOUS_IT_IMAGE_NAME-jdk${{ inputs.build_jdk }}-version${{ inputs.DRUID_PREVIOUS_VERSION }} + echo $DRUID_PREVIOUS_IT_IMAGE_NAME + docker save "$DRUID_PREVIOUS_IT_IMAGE_NAME" | gzip > druid-container-jdk${{ inputs.build_jdk }}-version${{ inputs.DRUID_PREVIOUS_VERSION }}.tar.gz + # unit-tests-phase2: # strategy: # fail-fast: false @@ -165,12 +165,12 @@ jobs: # uses: ./.github/workflows/standard-its.yml # # -# revised-its: -# needs: build -# if: true #${{ always() && (needs.unit-tests.result == 'success' || needs.unit-tests.outputs.continue_tests) }} -# uses: ./.github/workflows/revised-its.yml -# with: -# BACKWARD_COMPATIBILITY_IT_ENABLED: ${{ env.BACKWARD_COMPATIBILITY_IT_ENABLED }} -# DRUID_PREVIOUS_VERSION: ${{ env.DRUID_PREVIOUS_VERSION }} -# DRUID_PREVIOUS_VERSION_DOWNLOAD_URL: ${{ env.DRUID_PREVIOUS_VERSION_DOWNLOAD_URL }} -# DRUID_PREVIOUS_IT_IMAGE_NAME: ${{ env.DRUID_PREVIOUS_IT_IMAGE_NAME }} + revised-its: + needs: build + if: true #${{ always() && (needs.unit-tests.result == 'success' || needs.unit-tests.outputs.continue_tests) }} + uses: ./.github/workflows/revised-its.yml + with: + BACKWARD_COMPATIBILITY_IT_ENABLED: ${{ env.BACKWARD_COMPATIBILITY_IT_ENABLED }} + DRUID_PREVIOUS_VERSION: ${{ env.DRUID_PREVIOUS_VERSION }} + DRUID_PREVIOUS_VERSION_DOWNLOAD_URL: ${{ env.DRUID_PREVIOUS_VERSION_DOWNLOAD_URL }} + DRUID_PREVIOUS_IT_IMAGE_NAME: ${{ env.DRUID_PREVIOUS_IT_IMAGE_NAME }} From 984657f029d3e2d42d5f49b6cfb98610e0f7509a Mon Sep 17 00:00:00 2001 From: rishabh singh Date: Wed, 17 Jul 2024 12:53:47 +0530 Subject: [PATCH 24/31] Revert some changes --- .github/workflows/codeql.yml | 32 ++-- .github/workflows/cron-job-its.yml | 20 +-- .github/workflows/distribution-checks.yml | 34 ++-- .github/workflows/static-checks.yml | 149 ++---------------- .github/workflows/test-action.yml | 57 ------- .../unit-and-integration-tests-unified.yml | 123 +++++++-------- docs/ingestion/input-sources.md | 8 +- integration-tests-ex/image/docker-build.sh | 4 - .../AbstractSegmentMetadataCache.java | 2 - 9 files changed, 120 insertions(+), 309 deletions(-) delete mode 100644 .github/workflows/test-action.yml diff --git a/.github/workflows/codeql.yml b/.github/workflows/codeql.yml index 899f9e744d49..7cfeb63cbff0 100644 --- a/.github/workflows/codeql.yml +++ b/.github/workflows/codeql.yml @@ -1,21 +1,21 @@ name: "CodeQL" -#on: -# push: -# paths-ignore: -# - '**/*.md' -# - 'dev/**' -# - 'docs/**' -# branches: [ 'master', '0.6.x', '0.7.x', '0.7.1.x', '0.7.2.x', '0.8.0', '0.8.1', '0.8.2', '0.8.3', '0.9.0', '0.9.1' ] -# pull_request: -# paths-ignore: -# - '**/*.md' -# - 'dev/**' -# - 'docs/**' -# # The branches below must be a subset of the branches above -# branches: [ 'master' ] -# schedule: -# - cron: '18 15 * * 4' +on: + push: + paths-ignore: + - '**/*.md' + - 'dev/**' + - 'docs/**' + branches: [ 'master', '0.6.x', '0.7.x', '0.7.1.x', '0.7.2.x', '0.8.0', '0.8.1', '0.8.2', '0.8.3', '0.9.0', '0.9.1' ] + pull_request: + paths-ignore: + - '**/*.md' + - 'dev/**' + - 'docs/**' + # The branches below must be a subset of the branches above + branches: [ 'master' ] + schedule: + - cron: '18 15 * * 4' jobs: analyze: diff --git a/.github/workflows/cron-job-its.yml b/.github/workflows/cron-job-its.yml index 877fb2d385b6..aa3f28382884 100644 --- a/.github/workflows/cron-job-its.yml +++ b/.github/workflows/cron-job-its.yml @@ -14,16 +14,16 @@ # limitations under the License. name: "Cron Job ITs" -#on: -# schedule: # Runs by default on master branch -# - cron: '0 3 * * *' # Runs every day at 3:00 AM UTC -# pull_request: -# paths: -# - 'owasp-dependency-check-suppressions.xml' -# branches: -# - master -# - '[0-9]+.[0-9]+.[0-9]+' # release branches -# - '[0-9]+.[0-9]+.[0-9]+-[A-Za-z0-9]+' # release branches +on: + schedule: # Runs by default on master branch + - cron: '0 3 * * *' # Runs every day at 3:00 AM UTC + pull_request: + paths: + - 'owasp-dependency-check-suppressions.xml' + branches: + - master + - '[0-9]+.[0-9]+.[0-9]+' # release branches + - '[0-9]+.[0-9]+.[0-9]+-[A-Za-z0-9]+' # release branches jobs: build: diff --git a/.github/workflows/distribution-checks.yml b/.github/workflows/distribution-checks.yml index f0e6d2ef4626..caadb16b608e 100644 --- a/.github/workflows/distribution-checks.yml +++ b/.github/workflows/distribution-checks.yml @@ -14,23 +14,23 @@ # limitations under the License. name: "Distribution Checks" -#on: -# push: -# branches: -# - master -# - '[0-9]+.[0-9]+.[0-9]+' # release branches -# - '[0-9]+.[0-9]+.[0-9]+-[A-Za-z0-9]+' # release branches -# paths: -# - 'distribution/**' -# - '**/pom.xml' -# pull_request: -# branches: -# - master -# - '[0-9]+.[0-9]+.[0-9]+' # release branches -# - '[0-9]+.[0-9]+.[0-9]+-[A-Za-z0-9]+' # release branches -# paths: -# - 'distribution/**' -# - '**/pom.xml' +on: + push: + branches: + - master + - '[0-9]+.[0-9]+.[0-9]+' # release branches + - '[0-9]+.[0-9]+.[0-9]+-[A-Za-z0-9]+' # release branches + paths: + - 'distribution/**' + - '**/pom.xml' + pull_request: + branches: + - master + - '[0-9]+.[0-9]+.[0-9]+' # release branches + - '[0-9]+.[0-9]+.[0-9]+-[A-Za-z0-9]+' # release branches + paths: + - 'distribution/**' + - '**/pom.xml' jobs: docker-build: diff --git a/.github/workflows/static-checks.yml b/.github/workflows/static-checks.yml index 28005ae54c89..a374cf72ccfe 100644 --- a/.github/workflows/static-checks.yml +++ b/.github/workflows/static-checks.yml @@ -26,29 +26,22 @@ on: - '[0-9]+.[0-9]+.[0-9]+' # release branches - '[0-9]+.[0-9]+.[0-9]+-[A-Za-z0-9]+' # release branches -#concurrency: -# group: '${{ github.workflow }} @ ${{ github.event.pull_request.head.label || github.head_ref || github.run_id }}' # group workflows only on pull_requests and not on branch commits -# cancel-in-progress: true +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.pull_request.head.label || github.head_ref || github.run_id }}' # group workflows only on pull_requests and not on branch commits + cancel-in-progress: true env: MVN: mvn -B MAVEN_SKIP: -P skip-static-checks -Dweb.console.skip=true -Dmaven.javadoc.skip=true MAVEN_SKIP_TESTS: -P skip-tests MAVEN_OPTS: -Xmx8g - MYSQL_DRIVER_CLASSNAME: com.mysql.jdbc.Driver # Used to set druid config in docker image for revised ITs - SEGMENT_DOWNLOAD_TIMEOUT_MINS: 5 - BACKWARD_COMPATIBILITY_IT_ENABLED: false - DRUID_PREVIOUS_VERSION: 30.0.0 - DRUID_PREVIOUS_VERSION_DOWNLOAD_URL: https://dlcdn.apache.org/druid/30.0.0/apache-druid-30.0.0-bin.tar.gz - DRUID_PREVIOUS_IT_IMAGE_NAME: org.apache.druid.integration-tests/test:30.0.0 - jobs: static-checks: strategy: fail-fast: false matrix: - java: [ '8'] + java: [ '8', '11', '17', '21' ] runs-on: ubuntu-latest steps: - name: checkout branch @@ -61,7 +54,6 @@ jobs: cache: 'maven' - name: packaging check - if: false run: | ./.github/scripts/setup_generate_license.sh ${MVN} clean install -Prat --fail-at-end \ @@ -71,7 +63,7 @@ jobs: - name: script checks # who watches the watchers? - if: false #${{ matrix.java == '8' }} + if: ${{ matrix.java == '8' }} run: ./check_test_suite_test.py - name: (openjdk17) strict compilation @@ -88,43 +80,42 @@ jobs: ${MVN} install -q -ff -pl 'distribution' ${MAVEN_SKIP} ${MAVEN_SKIP_TESTS} - name: checkstyle - if: false #${{ matrix.java == '8' }} + if: ${{ matrix.java == '8' }} run: ${MVN} checkstyle:checkstyle --fail-at-end - name: license checks - if: false #${{ matrix.java == '8' }} + if: ${{ matrix.java == '8' }} run: ./.github/scripts/license_checks_script.sh - name: analyze dependencies - if: false #${{ matrix.java == '8' }} + if: ${{ matrix.java == '8' }} run: | ./.github/scripts/analyze_dependencies_script.sh - name: animal sniffer checks - if: false #${{ matrix.java == '8' }} + if: ${{ matrix.java == '8' }} run: ${MVN} animal-sniffer:check --fail-at-end - name: enforcer checks - if: false #${{ matrix.java == '8' }} + if: ${{ matrix.java == '8' }} run: ${MVN} enforcer:enforce --fail-at-end - name: forbidden api checks - if: false #${{ matrix.java == '8' }} + if: ${{ matrix.java == '8' }} run: ${MVN} forbiddenapis:check forbiddenapis:testCheck --fail-at-end - name: pmd checks - if: false #${{ matrix.java == '8' }} + if: ${{ matrix.java == '8' }} run: ${MVN} pmd:check --fail-at-end # TODO: consider adding pmd:cpd-check - name: spotbugs checks - if: false #${{ matrix.java == '8' }} + if: ${{ matrix.java == '8' }} run: ${MVN} spotbugs:check --fail-at-end -pl '!benchmarks' intellij-inspections: strategy: fail-fast: false runs-on: ubuntu-latest - if: false steps: - name: checkout branch uses: actions/checkout@v4 @@ -155,7 +146,6 @@ jobs: openrewrite: runs-on: ubuntu-latest - if: false steps: - name: checkout branch uses: actions/checkout@v4 @@ -188,7 +178,6 @@ jobs: strategy: fail-fast: false runs-on: ubuntu-latest - if: false steps: - name: checkout branch uses: actions/checkout@v4 @@ -237,115 +226,3 @@ jobs: with: name: Druid logs web-checks path: druid-logs.tgz - - - build: - name: "build (jdk${{ matrix.jdk }})" - strategy: - fail-fast: false - matrix: - jdk: [ '8' ] - runs-on: ubuntu-latest - steps: - - name: Checkout branch - uses: actions/checkout@v4 - - # skip the "cache: maven" step from setup-java. We explicitly use a - # different cache key since we cannot reuse it across commits. - - uses: actions/setup-java@v4 - with: - distribution: 'zulu' - java-version: ${{ matrix.jdk }} - - # the build step produces SNAPSHOT artifacts into the local maven repository, - # we include github.sha in the cache key to make it specific to that build/jdk - - name: Cache Maven m2 repository - id: maven - uses: actions/cache@v4 - with: - path: ~/.m2/repository - key: maven-${{ runner.os }}-${{ matrix.jdk }}-${{ github.sha }} - restore-keys: setup-java-Linux-maven-${{ hashFiles('**/pom.xml') }} - - - name: Cache targets - id: target - uses: actions/cache@v4 - with: - path: | - ./**/target - key: maven-${{ runner.os }}-${{ matrix.jdk }}-targets-${{ github.sha }} - - - name: Cache image - id: docker_container - uses: actions/cache@v4 - with: - key: druid-container-jdk${{ matrix.jdk }}.tar.gz-${{ github.sha }} - path: | - ./druid-container-jdk${{ matrix.jdk }}.tar.gz - ./integration-tests-ex/image/target/env.sh - - - name: Maven build - id: maven_build - run: | - ./it.sh ci - - - name: Container build - run: | - ./it.sh image - source ./integration-tests-ex/image/target/env.sh - docker tag $DRUID_IT_IMAGE_NAME $DRUID_IT_IMAGE_NAME-jdk${{ matrix.jdk }} - - - name: Save docker container to archive - run: | - source ./integration-tests-ex/image/target/env.sh - echo $DRUID_IT_IMAGE_NAME - docker save "$DRUID_IT_IMAGE_NAME" | gzip > druid-container-jdk${{ matrix.jdk }}.tar.gz - - - name: Save old version docker image - if: ${{ env.BACKWARD_COMPATIBILITY_IT_ENABLED == "true" }} - run: | - docker tag $DRUID_PREVIOUS_IT_IMAGE_NAME $DRUID_PREVIOUS_IT_IMAGE_NAME-jdk${{ inputs.build_jdk }}-version${{ inputs.DRUID_PREVIOUS_VERSION }} - echo $DRUID_PREVIOUS_IT_IMAGE_NAME - docker save "$DRUID_PREVIOUS_IT_IMAGE_NAME" | gzip > druid-container-jdk${{ inputs.build_jdk }}-version${{ inputs.DRUID_PREVIOUS_VERSION }}.tar.gz - - # unit-tests-phase2: - # strategy: - # fail-fast: false - # matrix: - # jdk: [ 11, 17, 21 ] - # name: "unit tests (jdk${{ matrix.jdk }}, sql-compat=true)" - # uses: ./.github/workflows/unit-tests.yml - # needs: unit-tests - # if: false #${{ always() && (needs.unit-tests.result == 'success' || needs.unit-tests.outputs.continue_tests) }} - # with: - # jdk: ${{ matrix.jdk }} - # sql_compatibility: true - # - # unit-tests: - # if: false - # strategy: - # fail-fast: false - # matrix: - # sql_compatibility: [ false, true ] - # name: "unit tests (jdk8, sql-compat=${{ matrix.sql_compatibility }})" - # uses: ./.github/workflows/unit-tests.yml - # needs: build - # with: - # jdk: 8 - # sql_compatibility: ${{ matrix.sql_compatibility }} - # - # standard-its: - # needs: unit-tests - # if: false #${{ always() && (needs.unit-tests.result == 'success' || needs.unit-tests.outputs.continue_tests) }} - # uses: ./.github/workflows/standard-its.yml - # - # - revised-its: - needs: build - if: true #${{ always() && (needs.unit-tests.result == 'success' || needs.unit-tests.outputs.continue_tests) }} - uses: ./.github/workflows/revised-its.yml - with: - BACKWARD_COMPATIBILITY_IT_ENABLED: ${{ env.BACKWARD_COMPATIBILITY_IT_ENABLED }} - DRUID_PREVIOUS_VERSION: ${{ env.DRUID_PREVIOUS_VERSION }} - DRUID_PREVIOUS_VERSION_DOWNLOAD_URL: ${{ env.DRUID_PREVIOUS_VERSION_DOWNLOAD_URL }} - DRUID_PREVIOUS_IT_IMAGE_NAME: ${{ env.DRUID_PREVIOUS_IT_IMAGE_NAME }} diff --git a/.github/workflows/test-action.yml b/.github/workflows/test-action.yml deleted file mode 100644 index c83e9d4baf3c..000000000000 --- a/.github/workflows/test-action.yml +++ /dev/null @@ -1,57 +0,0 @@ -# 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. - -name: "Test action" -#on: -# push: -# paths-ignore: -# - '**/*.md' -# - 'dev/**' -# - 'docs/**' -# - 'examples/**/jupyter-notebooks/**' -# - 'web-console/**' -# - 'website/**' -# branches: -# - master -# - '[0-9]+.[0-9]+.[0-9]+' # release branches -# - '[0-9]+.[0-9]+.[0-9]+-[A-Za-z0-9]+' # release branches -# pull_request: -# paths-ignore: -# - '**/*.md' -# - 'dev/**' -# - 'docs/**' -# - 'examples/**/jupyter-notebooks/**' -# - 'web-console/**' -# - 'website/**' -# branches: -# - master -# - '[0-9]+.[0-9]+.[0-9]+' # release branches -# - '[0-9]+.[0-9]+.[0-9]+-[A-Za-z0-9]+' # release branches -on: - # Trigger on push events to any branch - push: {} # Empty curly braces indicate all branches -# -#concurrency: -# group: '${{ github.workflow }} @ ${{ github.event.pull_request.head.label || github.head_ref || github.run_id }}' # group workflows only on pull_requests and not on branch commits -# cancel-in-progress: true - -jobs: - dummy-step: - runs-on: ubuntu-latest - name: "Dummy step" - steps: - - name: "Echo" - run: | - echo "run dummy-step" diff --git a/.github/workflows/unit-and-integration-tests-unified.yml b/.github/workflows/unit-and-integration-tests-unified.yml index 24a35702b902..d9eb2e7f1a5c 100644 --- a/.github/workflows/unit-and-integration-tests-unified.yml +++ b/.github/workflows/unit-and-integration-tests-unified.yml @@ -14,38 +14,36 @@ # limitations under the License. name: "Unit & Integration tests CI" -#on: -# push: -# paths-ignore: -# - '**/*.md' -# - 'dev/**' -# - 'docs/**' -# - 'examples/**/jupyter-notebooks/**' -# - 'web-console/**' -# - 'website/**' -# branches: -# - master -# - '[0-9]+.[0-9]+.[0-9]+' # release branches -# - '[0-9]+.[0-9]+.[0-9]+-[A-Za-z0-9]+' # release branches -# pull_request: -# paths-ignore: -# - '**/*.md' -# - 'dev/**' -# - 'docs/**' -# - 'examples/**/jupyter-notebooks/**' -# - 'web-console/**' -# - 'website/**' -# branches: -# - master -# - '[0-9]+.[0-9]+.[0-9]+' # release branches -# - '[0-9]+.[0-9]+.[0-9]+-[A-Za-z0-9]+' # release branches on: - # Trigger on push events to any branch - push: {} # Empty curly braces indicate all branches -# -#concurrency: -# group: '${{ github.workflow }} @ ${{ github.event.pull_request.head.label || github.head_ref || github.run_id }}' # group workflows only on pull_requests and not on branch commits -# cancel-in-progress: true + push: + paths-ignore: + - '**/*.md' + - 'dev/**' + - 'docs/**' + - 'examples/**/jupyter-notebooks/**' + - 'web-console/**' + - 'website/**' + branches: + - master + - '[0-9]+.[0-9]+.[0-9]+' # release branches + - '[0-9]+.[0-9]+.[0-9]+-[A-Za-z0-9]+' # release branches + pull_request: + paths-ignore: + - '**/*.md' + - 'dev/**' + - 'docs/**' + - 'examples/**/jupyter-notebooks/**' + - 'web-console/**' + - 'website/**' + branches: + - master + - '[0-9]+.[0-9]+.[0-9]+' # release branches + - '[0-9]+.[0-9]+.[0-9]+-[A-Za-z0-9]+' # release branches + + +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.pull_request.head.label || github.head_ref || github.run_id }}' # group workflows only on pull_requests and not on branch commits + cancel-in-progress: true env: MYSQL_DRIVER_CLASSNAME: com.mysql.jdbc.Driver # Used to set druid config in docker image for revised ITs @@ -133,38 +131,37 @@ jobs: echo $DRUID_PREVIOUS_IT_IMAGE_NAME docker save "$DRUID_PREVIOUS_IT_IMAGE_NAME" | gzip > druid-container-jdk${{ inputs.build_jdk }}-version${{ inputs.DRUID_PREVIOUS_VERSION }}.tar.gz -# unit-tests-phase2: -# strategy: -# fail-fast: false -# matrix: -# jdk: [ 11, 17, 21 ] -# name: "unit tests (jdk${{ matrix.jdk }}, sql-compat=true)" -# uses: ./.github/workflows/unit-tests.yml -# needs: unit-tests -# if: false #${{ always() && (needs.unit-tests.result == 'success' || needs.unit-tests.outputs.continue_tests) }} -# with: -# jdk: ${{ matrix.jdk }} -# sql_compatibility: true -# -# unit-tests: -# if: false -# strategy: -# fail-fast: false -# matrix: -# sql_compatibility: [ false, true ] -# name: "unit tests (jdk8, sql-compat=${{ matrix.sql_compatibility }})" -# uses: ./.github/workflows/unit-tests.yml -# needs: build -# with: -# jdk: 8 -# sql_compatibility: ${{ matrix.sql_compatibility }} -# -# standard-its: -# needs: unit-tests -# if: false #${{ always() && (needs.unit-tests.result == 'success' || needs.unit-tests.outputs.continue_tests) }} -# uses: ./.github/workflows/standard-its.yml -# -# + unit-tests-phase2: + strategy: + fail-fast: false + matrix: + jdk: [ 11, 17, 21 ] + name: "unit tests (jdk${{ matrix.jdk }}, sql-compat=true)" + uses: ./.github/workflows/unit-tests.yml + needs: unit-tests + if: false #${{ always() && (needs.unit-tests.result == 'success' || needs.unit-tests.outputs.continue_tests) }} + with: + jdk: ${{ matrix.jdk }} + sql_compatibility: true + + unit-tests: + if: false + strategy: + fail-fast: false + matrix: + sql_compatibility: [ false, true ] + name: "unit tests (jdk8, sql-compat=${{ matrix.sql_compatibility }})" + uses: ./.github/workflows/unit-tests.yml + needs: build + with: + jdk: 8 + sql_compatibility: ${{ matrix.sql_compatibility }} + + standard-its: + needs: unit-tests + if: false #${{ always() && (needs.unit-tests.result == 'success' || needs.unit-tests.outputs.continue_tests) }} + uses: ./.github/workflows/standard-its.yml + revised-its: needs: build if: true #${{ always() && (needs.unit-tests.result == 'success' || needs.unit-tests.outputs.continue_tests) }} diff --git a/docs/ingestion/input-sources.md b/docs/ingestion/input-sources.md index 6d8434d5d537..fb8e1f98c91f 100644 --- a/docs/ingestion/input-sources.md +++ b/docs/ingestion/input-sources.md @@ -87,7 +87,7 @@ Sample specs: "type": "s3", "objectGlob": "**.json", "objects": [ - { "bucket": "foo", "path": "bar/file1.yml.json"}, + { "bucket": "foo", "path": "bar/file1.json"}, { "bucket": "bar", "path": "foo/file2.json"} ] }, @@ -275,7 +275,7 @@ Sample specs: "type": "google", "objectGlob": "**.json", "objects": [ - { "bucket": "foo", "path": "bar/file1.yml.json"}, + { "bucket": "foo", "path": "bar/file1.json"}, { "bucket": "bar", "path": "foo/file2.json"} ] }, @@ -362,7 +362,7 @@ Sample specs: "type": "azureStorage", "objectGlob": "**.json", "objects": [ - { "bucket": "storageAccount", "path": "container/prefix1/file1.yml.json"}, + { "bucket": "storageAccount", "path": "container/prefix1/file1.json"}, { "bucket": "storageAccount", "path": "container/prefix2/file2.json"} ], "properties": { @@ -459,7 +459,7 @@ Sample specs: "type": "azure", "objectGlob": "**.json", "objects": [ - { "bucket": "container", "path": "prefix1/file1.yml.json"}, + { "bucket": "container", "path": "prefix1/file1.json"}, { "bucket": "container", "path": "prefix2/file2.json"} ] }, diff --git a/integration-tests-ex/image/docker-build.sh b/integration-tests-ex/image/docker-build.sh index de806621a7da..c47513860e34 100755 --- a/integration-tests-ex/image/docker-build.sh +++ b/integration-tests-ex/image/docker-build.sh @@ -26,8 +26,6 @@ SCRIPT_DIR=$(cd $(dirname $0) && pwd) -echo "script_dir is $SCRIPT_DIR" - # Maven should have created the docker dir with the needed # dependency jars. If doing this by hand, run Maven once to # populate these jars. @@ -62,8 +60,6 @@ if [ $BACKWARD_COMPATIBILITY_IT_ENABLED != "true" ]; then exit 1 fi -echo "Building previous docker image" - # Download the previous druid tar curl -L $DRUID_PREVIOUS_VERSION_DOWNLOAD_URL --output apache-druid-$DRUID_PREV_VERSION-bin.tar.gz diff --git a/server/src/main/java/org/apache/druid/segment/metadata/AbstractSegmentMetadataCache.java b/server/src/main/java/org/apache/druid/segment/metadata/AbstractSegmentMetadataCache.java index 41ce236832d2..9cb2297db828 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/AbstractSegmentMetadataCache.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/AbstractSegmentMetadataCache.java @@ -106,8 +106,6 @@ */ public abstract class AbstractSegmentMetadataCache { - - private static final EmittingLogger log = new EmittingLogger(AbstractSegmentMetadataCache.class); private static final int MAX_SEGMENTS_PER_QUERY = 15000; private static final long DEFAULT_NUM_ROWS = 0; From 86ef56f5d5d9c8e6df897c3c055b2625e61939d8 Mon Sep 17 00:00:00 2001 From: rishabh singh Date: Wed, 17 Jul 2024 14:49:34 +0530 Subject: [PATCH 25/31] Add test under BackwardCompatibility group --- .../ITBackwardCompatibilityIndexerTest.java | 14 + .../druid/testsEx/indexer/ITIndexerTest.java | 381 +---------------- .../druid/testsEx/indexer/IndexerTest.java | 384 ++++++++++++++++++ .../leadership/HighAvailabilityTest.java | 241 ----------- .../leadership/ITHighAvailabilityTest.java | 237 ++++++++++- 5 files changed, 635 insertions(+), 622 deletions(-) create mode 100644 integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/BackwardCompatibility/ITBackwardCompatibilityIndexerTest.java create mode 100644 integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/IndexerTest.java delete mode 100644 integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/leadership/HighAvailabilityTest.java diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/BackwardCompatibility/ITBackwardCompatibilityIndexerTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/BackwardCompatibility/ITBackwardCompatibilityIndexerTest.java new file mode 100644 index 000000000000..5945148b9bcc --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/BackwardCompatibility/ITBackwardCompatibilityIndexerTest.java @@ -0,0 +1,14 @@ +package org.apache.druid.testsEx.BackwardCompatibility; + +import org.apache.druid.testsEx.categories.BackwardCompatibility; +import org.apache.druid.testsEx.categories.BatchIndex; +import org.apache.druid.testsEx.config.DruidTestRunner; +import org.apache.druid.testsEx.indexer.IndexerTest; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; + +@RunWith(DruidTestRunner.class) +@Category({BackwardCompatibility.class}) +public class ITBackwardCompatibilityIndexerTest extends IndexerTest +{ +} 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 index ce89c5f135a1..8167aa3ac8d4 100644 --- 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 @@ -1,389 +1,12 @@ -/* - * 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.BackwardCompatibility; import org.apache.druid.testsEx.categories.BatchIndex; -import org.apache.druid.testsEx.categories.InputSource; 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({BackwardCompatibility.class}) -public class ITIndexerTest extends AbstractITBatchIndexTest +@Category({BatchIndex.class}) +public class ITIndexerTest extends IndexerTest { - - 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/IndexerTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/IndexerTest.java new file mode 100644 index 000000000000..50eeb511c1c7 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/IndexerTest.java @@ -0,0 +1,384 @@ +/* + * 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.BackwardCompatibility; +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; + +public class IndexerTest 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/leadership/HighAvailabilityTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/leadership/HighAvailabilityTest.java deleted file mode 100644 index 765b9ac822ee..000000000000 --- a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/leadership/HighAvailabilityTest.java +++ /dev/null @@ -1,241 +0,0 @@ -/* - * 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.testsEx.categories.HighAvailability; -import org.apache.druid.testsEx.cluster.DruidClusterClient; -import org.apache.druid.testsEx.config.Initializer; -import org.apache.druid.testsEx.indexer.AbstractIndexerTest; -import org.apache.druid.testsEx.utils.DruidClusterAdminClient; -import org.junit.Test; - -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; - -public class HighAvailabilityTest -{ - private static final Logger LOG = new Logger(HighAvailabilityTest.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/leadership/ITHighAvailabilityTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/leadership/ITHighAvailabilityTest.java index 42f633c3ff42..60d22f3f9f8f 100644 --- 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 @@ -1,13 +1,246 @@ -package org.apache.druid.testsEx.leadership; +/* + * 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.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.indexer.AbstractIndexerTest; +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 extends HighAvailabilityTest +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()); + } } From 96f212bdc35f30a7028f9bd63707bdd7e5a4244e Mon Sep 17 00:00:00 2001 From: rishabh singh Date: Thu, 18 Jul 2024 22:16:16 +0530 Subject: [PATCH 26/31] debug unified it script --- .../unit-and-integration-tests-unified.yml | 32 +++++++------------ 1 file changed, 12 insertions(+), 20 deletions(-) diff --git a/.github/workflows/unit-and-integration-tests-unified.yml b/.github/workflows/unit-and-integration-tests-unified.yml index d9eb2e7f1a5c..e5745bbda56c 100644 --- a/.github/workflows/unit-and-integration-tests-unified.yml +++ b/.github/workflows/unit-and-integration-tests-unified.yml @@ -54,14 +54,6 @@ env: DRUID_PREVIOUS_IT_IMAGE_NAME: org.apache.druid.integration-tests/test:30.0.0 jobs: - dummy-step: - runs-on: ubuntu-latest - name: "Dummy step" - steps: - - name: "Echo" - run: | - echo "run dummy-step" - build: name: "build (jdk${{ matrix.jdk }})" strategy: @@ -124,12 +116,12 @@ jobs: echo $DRUID_IT_IMAGE_NAME docker save "$DRUID_IT_IMAGE_NAME" | gzip > druid-container-jdk${{ matrix.jdk }}.tar.gz - - name: Save old version docker image - if: ${{ env.BACKWARD_COMPATIBILITY_IT_ENABLED == "true" }} - run: | - docker tag $DRUID_PREVIOUS_IT_IMAGE_NAME $DRUID_PREVIOUS_IT_IMAGE_NAME-jdk${{ inputs.build_jdk }}-version${{ inputs.DRUID_PREVIOUS_VERSION }} - echo $DRUID_PREVIOUS_IT_IMAGE_NAME - docker save "$DRUID_PREVIOUS_IT_IMAGE_NAME" | gzip > druid-container-jdk${{ inputs.build_jdk }}-version${{ inputs.DRUID_PREVIOUS_VERSION }}.tar.gz +# - name: Save old version docker image +# if: ${{ env.BACKWARD_COMPATIBILITY_IT_ENABLED == "true" }} +# run: | +# docker tag $DRUID_PREVIOUS_IT_IMAGE_NAME $DRUID_PREVIOUS_IT_IMAGE_NAME-jdk${{ inputs.build_jdk }}-version${{ inputs.DRUID_PREVIOUS_VERSION }} +# echo $DRUID_PREVIOUS_IT_IMAGE_NAME +# docker save "$DRUID_PREVIOUS_IT_IMAGE_NAME" | gzip > druid-container-jdk${{ inputs.build_jdk }}-version${{ inputs.DRUID_PREVIOUS_VERSION }}.tar.gz unit-tests-phase2: strategy: @@ -164,10 +156,10 @@ jobs: revised-its: needs: build - if: true #${{ always() && (needs.unit-tests.result == 'success' || needs.unit-tests.outputs.continue_tests) }} + if: false #${{ always() && (needs.unit-tests.result == 'success' || needs.unit-tests.outputs.continue_tests) }} uses: ./.github/workflows/revised-its.yml - with: - BACKWARD_COMPATIBILITY_IT_ENABLED: ${{ env.BACKWARD_COMPATIBILITY_IT_ENABLED }} - DRUID_PREVIOUS_VERSION: ${{ env.DRUID_PREVIOUS_VERSION }} - DRUID_PREVIOUS_VERSION_DOWNLOAD_URL: ${{ env.DRUID_PREVIOUS_VERSION_DOWNLOAD_URL }} - DRUID_PREVIOUS_IT_IMAGE_NAME: ${{ env.DRUID_PREVIOUS_IT_IMAGE_NAME }} +# with: +# BACKWARD_COMPATIBILITY_IT_ENABLED: ${{ env.BACKWARD_COMPATIBILITY_IT_ENABLED }} +# DRUID_PREVIOUS_VERSION: ${{ env.DRUID_PREVIOUS_VERSION }} +# DRUID_PREVIOUS_VERSION_DOWNLOAD_URL: ${{ env.DRUID_PREVIOUS_VERSION_DOWNLOAD_URL }} +# DRUID_PREVIOUS_IT_IMAGE_NAME: ${{ env.DRUID_PREVIOUS_IT_IMAGE_NAME }} From b02967275c868f4c2bdf5a1dca8eed6e0b638ef7 Mon Sep 17 00:00:00 2001 From: rishabh singh Date: Thu, 18 Jul 2024 22:20:22 +0530 Subject: [PATCH 27/31] static check failures --- .../ITBackwardCompatibilityIndexerTest.java | 20 ++++++++++++++++++- .../categories/BackwardCompatibility.java | 19 ++++++++++++++++++ .../druid/testsEx/indexer/ITIndexerTest.java | 19 ++++++++++++++++++ 3 files changed, 57 insertions(+), 1 deletion(-) diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/BackwardCompatibility/ITBackwardCompatibilityIndexerTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/BackwardCompatibility/ITBackwardCompatibilityIndexerTest.java index 5945148b9bcc..7d07d518922e 100644 --- a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/BackwardCompatibility/ITBackwardCompatibilityIndexerTest.java +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/BackwardCompatibility/ITBackwardCompatibilityIndexerTest.java @@ -1,7 +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.BackwardCompatibility; import org.apache.druid.testsEx.categories.BackwardCompatibility; -import org.apache.druid.testsEx.categories.BatchIndex; import org.apache.druid.testsEx.config.DruidTestRunner; import org.apache.druid.testsEx.indexer.IndexerTest; import org.junit.experimental.categories.Category; diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/categories/BackwardCompatibility.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/categories/BackwardCompatibility.java index 8ce979fe4a2a..7e357e6df8f0 100644 --- a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/categories/BackwardCompatibility.java +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/categories/BackwardCompatibility.java @@ -1,3 +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. + */ + package org.apache.druid.testsEx.categories; public class BackwardCompatibility 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 index 8167aa3ac8d4..06a097d608f9 100644 --- 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 @@ -1,3 +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. + */ + package org.apache.druid.testsEx.indexer; import org.apache.druid.testsEx.categories.BatchIndex; From 8ee9f2affa1ba0bf881e7ffb443fffb912a8327f Mon Sep 17 00:00:00 2001 From: rishabh singh Date: Thu, 18 Jul 2024 22:23:39 +0530 Subject: [PATCH 28/31] Debug GHA: restrict static-checks to run on only java8 --- .github/workflows/static-checks.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/static-checks.yml b/.github/workflows/static-checks.yml index a374cf72ccfe..979e03ffbff6 100644 --- a/.github/workflows/static-checks.yml +++ b/.github/workflows/static-checks.yml @@ -41,7 +41,7 @@ jobs: strategy: fail-fast: false matrix: - java: [ '8', '11', '17', '21' ] + java: [ '8' ] runs-on: ubuntu-latest steps: - name: checkout branch From 35c838d4923aa3a2bd6f543afc90515192c16f4d Mon Sep 17 00:00:00 2001 From: rishabh singh Date: Fri, 19 Jul 2024 01:25:29 +0530 Subject: [PATCH 29/31] test --- .github/workflows/unit-and-integration-tests-unified.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/unit-and-integration-tests-unified.yml b/.github/workflows/unit-and-integration-tests-unified.yml index e5745bbda56c..fd0d6c9c1948 100644 --- a/.github/workflows/unit-and-integration-tests-unified.yml +++ b/.github/workflows/unit-and-integration-tests-unified.yml @@ -156,7 +156,7 @@ jobs: revised-its: needs: build - if: false #${{ always() && (needs.unit-tests.result == 'success' || needs.unit-tests.outputs.continue_tests) }} + if: true #${{ always() && (needs.unit-tests.result == 'success' || needs.unit-tests.outputs.continue_tests) }} uses: ./.github/workflows/revised-its.yml # with: # BACKWARD_COMPATIBILITY_IT_ENABLED: ${{ env.BACKWARD_COMPATIBILITY_IT_ENABLED }} From 86feba42eee5636d004322b101b611017712e295 Mon Sep 17 00:00:00 2001 From: rishabh singh Date: Fri, 19 Jul 2024 01:26:28 +0530 Subject: [PATCH 30/31] test --- .github/workflows/unit-and-integration-tests-unified.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/unit-and-integration-tests-unified.yml b/.github/workflows/unit-and-integration-tests-unified.yml index fd0d6c9c1948..c7e2dcf9756b 100644 --- a/.github/workflows/unit-and-integration-tests-unified.yml +++ b/.github/workflows/unit-and-integration-tests-unified.yml @@ -156,7 +156,7 @@ jobs: revised-its: needs: build - if: true #${{ always() && (needs.unit-tests.result == 'success' || needs.unit-tests.outputs.continue_tests) }} + if: ${{ always() }} #${{ always() && (needs.unit-tests.result == 'success' || needs.unit-tests.outputs.continue_tests) }} uses: ./.github/workflows/revised-its.yml # with: # BACKWARD_COMPATIBILITY_IT_ENABLED: ${{ env.BACKWARD_COMPATIBILITY_IT_ENABLED }} From 133edfea7d6db4baa50b2b45f91646f82ece400e Mon Sep 17 00:00:00 2001 From: rishabh singh Date: Fri, 19 Jul 2024 01:27:21 +0530 Subject: [PATCH 31/31] test --- .github/workflows/unit-and-integration-tests-unified.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/unit-and-integration-tests-unified.yml b/.github/workflows/unit-and-integration-tests-unified.yml index c7e2dcf9756b..de5fa0821b95 100644 --- a/.github/workflows/unit-and-integration-tests-unified.yml +++ b/.github/workflows/unit-and-integration-tests-unified.yml @@ -155,7 +155,7 @@ jobs: uses: ./.github/workflows/standard-its.yml revised-its: - needs: build + needs: unit-tests if: ${{ always() }} #${{ always() && (needs.unit-tests.result == 'success' || needs.unit-tests.outputs.continue_tests) }} uses: ./.github/workflows/revised-its.yml # with: