You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by xi...@apache.org on 2023/07/20 01:04:20 UTC

[pinot] branch master updated: Allow each test to specify the query engine (#11138)

This is an automated email from the ASF dual-hosted git repository.

xiangfu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/pinot.git


The following commit(s) were added to refs/heads/master by this push:
     new 961695798b Allow each test to specify the query engine (#11138)
961695798b is described below

commit 961695798ba143101b6b4bd90b0cc8f61593d35c
Author: Xiang Fu <xi...@gmail.com>
AuthorDate: Wed Jul 19 18:04:14 2023 -0700

    Allow each test to specify the query engine (#11138)
---
 .../pinot/integration/tests/ClusterTest.java       | 39 +++++++++++++++-
 .../tests/JsonPathClusterIntegrationTest.java      | 52 +++++++++++++++++-----
 .../tests/MapTypeClusterIntegrationTest.java       | 24 +++++++---
 ...iStageEngineJsonPathClusterIntegrationTest.java | 52 ----------------------
 ...tiStageEngineMapTypeClusterIntegrationTest.java | 37 ---------------
 5 files changed, 95 insertions(+), 109 deletions(-)

diff --git a/pinot-integration-test-base/src/test/java/org/apache/pinot/integration/tests/ClusterTest.java b/pinot-integration-test-base/src/test/java/org/apache/pinot/integration/tests/ClusterTest.java
index f2d6e865a3..42988b57e7 100644
--- a/pinot-integration-test-base/src/test/java/org/apache/pinot/integration/tests/ClusterTest.java
+++ b/pinot-integration-test-base/src/test/java/org/apache/pinot/integration/tests/ClusterTest.java
@@ -76,6 +76,7 @@ import org.apache.pinot.spi.utils.JsonUtils;
 import org.apache.pinot.spi.utils.NetUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.testng.annotations.DataProvider;
 
 import static org.testng.Assert.assertEquals;
 import static org.testng.Assert.assertNotNull;
@@ -96,12 +97,26 @@ public abstract class ClusterTest extends ControllerTest {
 
   private String _brokerBaseApiUrl;
 
+  private boolean _useMultiStageQueryEngine = false;
+
   protected String getBrokerBaseApiUrl() {
     return _brokerBaseApiUrl;
   }
 
   protected boolean useMultiStageQueryEngine() {
-    return false;
+    return _useMultiStageQueryEngine;
+  }
+
+  protected void setUseMultiStageQueryEngine(boolean useMultiStageQueryEngine) {
+    _useMultiStageQueryEngine = useMultiStageQueryEngine;
+  }
+
+  protected void disableMultiStageQueryEngine() {
+    setUseMultiStageQueryEngine(false);
+  }
+
+  protected void enableMultiStageQueryEngine() {
+    setUseMultiStageQueryEngine(true);
   }
 
   protected PinotConfiguration getDefaultBrokerConfiguration() {
@@ -490,4 +505,26 @@ public abstract class ClusterTest extends ControllerTest {
     return JsonUtils.stringToJsonNode(
         sendPostRequest(controllerBaseApiUrl + "/sql", JsonUtils.objectToString(payload), headers));
   }
+
+  @DataProvider(name = "useBothQueryEngines")
+  public Object[][] useBothQueryEngines() {
+    return new Object[][]{
+        {false},
+        {true}
+    };
+  }
+
+  @DataProvider(name = "useV1QueryEngine")
+  public Object[][] useV1QueryEngine() {
+    return new Object[][]{
+        {false}
+    };
+  }
+
+  @DataProvider(name = "useV2QueryEngine")
+  public Object[][] useV2QueryEngine() {
+    return new Object[][]{
+        {true}
+    };
+  }
 }
diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/JsonPathClusterIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/JsonPathClusterIntegrationTest.java
index c0906d867a..f6a8f23826 100644
--- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/JsonPathClusterIntegrationTest.java
+++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/JsonPathClusterIntegrationTest.java
@@ -144,9 +144,11 @@ public class JsonPathClusterIntegrationTest extends BaseClusterIntegrationTest {
     return avroFile;
   }
 
-  @Test
-  public void testQueries()
+  @Test(dataProvider = "useBothQueryEngines")
+  public void testQueries(boolean useMultiStageQueryEngine)
       throws Exception {
+    setUseMultiStageQueryEngine(useMultiStageQueryEngine);
+
     //Selection Query
     String query = "Select myMapStr from " + DEFAULT_TABLE_NAME;
     JsonNode pinotResponse = postQuery(query);
@@ -194,9 +196,10 @@ public class JsonPathClusterIntegrationTest extends BaseClusterIntegrationTest {
     }
   }
 
-  @Test
-  public void testComplexQueries()
+  @Test(dataProvider = "useBothQueryEngines")
+  public void testComplexQueries(boolean useMultiStageQueryEngine)
       throws Exception {
+    setUseMultiStageQueryEngine(useMultiStageQueryEngine);
     //Selection Query
     String query = "Select complexMapStr from " + DEFAULT_TABLE_NAME;
     JsonNode pinotResponse = postQuery(query);
@@ -262,9 +265,11 @@ public class JsonPathClusterIntegrationTest extends BaseClusterIntegrationTest {
     }
   }
 
-  @Test
-  public void testComplexGroupByQuery()
+  @Test(dataProvider = "useV1QueryEngine")
+  public void testComplexGroupByQueryV1(boolean useMultiStageQueryEngine)
       throws Exception {
+    setUseMultiStageQueryEngine(useMultiStageQueryEngine);
+
     //Group By Query
     String query = "Select" + " jsonExtractScalar(complexMapStr,'$.k1','STRING'),"
         + " sum(jsonExtractScalar(complexMapStr,'$.k4.met','INT'))" + " from " + DEFAULT_TABLE_NAME
@@ -281,9 +286,30 @@ public class JsonPathClusterIntegrationTest extends BaseClusterIntegrationTest {
     }
   }
 
-  @Test
-  public void testQueryWithIntegerDefault()
+  @Test(dataProvider = "useV2QueryEngine")
+  public void testComplexGroupByQueryV2(boolean useMultiStageQueryEngine)
+      throws Exception {
+    setUseMultiStageQueryEngine(useMultiStageQueryEngine);
+    //Group By Query
+    String query = "Select" + " jsonExtractScalar(complexMapStr,'$.k1','STRING'),"
+        + " sum(jsonExtractScalar(complexMapStr,'$.k4.met','INT'))" + " from " + DEFAULT_TABLE_NAME
+        + " group by jsonExtractScalar(complexMapStr,'$.k1','STRING')"
+        + " order by sum(jsonExtractScalar(complexMapStr,'$.k4.met','INT')) DESC";
+    JsonNode pinotResponse = postQuery(query);
+    Assert.assertNotNull(pinotResponse.get("resultTable").get("rows"));
+    ArrayNode rows = (ArrayNode) pinotResponse.get("resultTable").get("rows");
+    for (int i = 0; i < rows.size(); i++) {
+      String seqId = String.valueOf(NUM_TOTAL_DOCS - 1 - i);
+      final JsonNode row = rows.get(i);
+      Assert.assertEquals(row.get(0).asText(), "value-k1-" + seqId);
+      Assert.assertEquals(row.get(1).asDouble(), Double.parseDouble(seqId));
+    }
+  }
+
+  @Test(dataProvider = "useBothQueryEngines")
+  public void testQueryWithIntegerDefault(boolean useMultiStageQueryEngine)
       throws Exception {
+    setUseMultiStageQueryEngine(useMultiStageQueryEngine);
     //Group By Query
     String query = "Select" + " jsonExtractScalar(complexMapStr,'$.inExistKey','STRING','defaultKey'),"
         + " sum(jsonExtractScalar(complexMapStr,'$.inExistMet','INT','1'))" + " from " + DEFAULT_TABLE_NAME
@@ -298,9 +324,10 @@ public class JsonPathClusterIntegrationTest extends BaseClusterIntegrationTest {
     Assert.assertEquals(row.get(1).asDouble(), 1000.0);
   }
 
-  @Test
-  public void testQueryWithDoubleDefault()
+  @Test(dataProvider = "useBothQueryEngines")
+  public void testQueryWithDoubleDefault(boolean useMultiStageQueryEngine)
       throws Exception {
+    setUseMultiStageQueryEngine(useMultiStageQueryEngine);
     //Group By Query
     String query = "Select" + " jsonExtractScalar(complexMapStr,'$.inExistKey','STRING', 'defaultKey'),"
         + " sum(jsonExtractScalar(complexMapStr,'$.inExistMet','DOUBLE','0.1'))" + " from " + DEFAULT_TABLE_NAME
@@ -315,9 +342,10 @@ public class JsonPathClusterIntegrationTest extends BaseClusterIntegrationTest {
     Assert.assertTrue(Math.abs(row.get(1).asDouble() - 100.0) < 1e-10);
   }
 
-  @Test
-  void testFailedQuery()
+  @Test(dataProvider = "useBothQueryEngines")
+  void testFailedQuery(boolean useMultiStageQueryEngine)
       throws Exception {
+    setUseMultiStageQueryEngine(useMultiStageQueryEngine);
     String query = "Select jsonExtractScalar(myMapStr,\"$.k1\",\"STRING\") from " + DEFAULT_TABLE_NAME;
     JsonNode pinotResponse = postQuery(query);
     Assert.assertEquals(pinotResponse.get("exceptions").get(0).get("errorCode").asInt(), 150);
diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MapTypeClusterIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MapTypeClusterIntegrationTest.java
index 3a4565abad..51724325d0 100644
--- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MapTypeClusterIntegrationTest.java
+++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MapTypeClusterIntegrationTest.java
@@ -53,6 +53,7 @@ public class MapTypeClusterIntegrationTest extends BaseClusterIntegrationTest {
   private static final String INT_KEY_MAP_FIELD_NAME = "intKeyMap";
   private static final String STRING_KEY_MAP_STR_FIELD_NAME = "stringKeyMapStr";
   private static final String INT_KEY_MAP_STR_FIELD_NAME = "intKeyMapStr";
+  private int _setSelectionDefaultDocCount = 10;
 
   @Override
   protected long getCountStarResult() {
@@ -131,12 +132,13 @@ public class MapTypeClusterIntegrationTest extends BaseClusterIntegrationTest {
   }
 
   protected int getSelectionDefaultDocCount() {
-    return 10;
+    return _setSelectionDefaultDocCount;
   }
 
-  @Test
-  public void testJsonPathQueries()
+  @Test(dataProvider = "useBothQueryEngines")
+  public void testJsonPathQueries(boolean useMultiStageQueryEngine)
       throws Exception {
+    setUseMultiStageQueryEngine(useMultiStageQueryEngine);
     // Selection only
     String query = "SELECT stringKeyMapStr FROM " + getTableName();
     JsonNode pinotResponse = postQuery(query);
@@ -252,9 +254,10 @@ public class MapTypeClusterIntegrationTest extends BaseClusterIntegrationTest {
     assertEquals(pinotResponse.get("exceptions").size(), 0);
   }
 
-  @Test
-  public void testQueries()
+  @Test(dataProvider = "useBothQueryEngines")
+  public void testQueries(boolean useMultiStageQueryEngine)
       throws Exception {
+    setUseMultiStageQueryEngine(useMultiStageQueryEngine);
     // Selection only
     String query = "SELECT mapValue(stringKeyMap__KEYS, 'k1', stringKeyMap__VALUES) FROM " + getTableName();
     JsonNode pinotResponse = postQuery(query);
@@ -368,9 +371,10 @@ public class MapTypeClusterIntegrationTest extends BaseClusterIntegrationTest {
     assertNotEquals(pinotResponse.get("exceptions").size(), 0);
   }
 
-  @Test
-  public void testMultiValueQueries()
+  @Test(dataProvider = "useV1QueryEngine")
+  public void testMultiValueQueries(boolean useMultiStageQueryEngine)
       throws Exception {
+    setUseMultiStageQueryEngine(useMultiStageQueryEngine);
     String query;
     JsonNode pinotResponse;
     JsonNode rows;
@@ -431,4 +435,10 @@ public class MapTypeClusterIntegrationTest extends BaseClusterIntegrationTest {
 
     FileUtils.deleteDirectory(_tempDir);
   }
+
+  @Override
+  protected void setUseMultiStageQueryEngine(boolean useMultiStageQueryEngine) {
+    super.setUseMultiStageQueryEngine(useMultiStageQueryEngine);
+    _setSelectionDefaultDocCount = useMultiStageQueryEngine ? 1000 : 10;
+  }
 }
diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MultiStageEngineJsonPathClusterIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MultiStageEngineJsonPathClusterIntegrationTest.java
deleted file mode 100644
index bbf996d046..0000000000
--- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MultiStageEngineJsonPathClusterIntegrationTest.java
+++ /dev/null
@@ -1,52 +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.pinot.integration.tests;
-
-import com.fasterxml.jackson.databind.JsonNode;
-import com.fasterxml.jackson.databind.node.ArrayNode;
-import org.testng.Assert;
-import org.testng.annotations.Test;
-
-
-public class MultiStageEngineJsonPathClusterIntegrationTest extends JsonPathClusterIntegrationTest {
-
-  @Override
-  protected boolean useMultiStageQueryEngine() {
-    return true;
-  }
-
-  @Test
-  public void testComplexGroupByQuery()
-      throws Exception {
-    //Group By Query
-    String query = "Select" + " jsonExtractScalar(complexMapStr,'$.k1','STRING'),"
-        + " sum(jsonExtractScalar(complexMapStr,'$.k4.met','INT'))" + " from " + DEFAULT_TABLE_NAME
-        + " group by jsonExtractScalar(complexMapStr,'$.k1','STRING')"
-        + " order by sum(jsonExtractScalar(complexMapStr,'$.k4.met','INT')) DESC";
-    JsonNode pinotResponse = postQuery(query);
-    Assert.assertNotNull(pinotResponse.get("resultTable").get("rows"));
-    ArrayNode rows = (ArrayNode) pinotResponse.get("resultTable").get("rows");
-    for (int i = 0; i < rows.size(); i++) {
-      String seqId = String.valueOf(NUM_TOTAL_DOCS - 1 - i);
-      final JsonNode row = rows.get(i);
-      Assert.assertEquals(row.get(0).asText(), "value-k1-" + seqId);
-      Assert.assertEquals(row.get(1).asDouble(), Double.parseDouble(seqId));
-    }
-  }
-}
diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MultiStageEngineMapTypeClusterIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MultiStageEngineMapTypeClusterIntegrationTest.java
deleted file mode 100644
index 62cfe12e00..0000000000
--- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MultiStageEngineMapTypeClusterIntegrationTest.java
+++ /dev/null
@@ -1,37 +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.pinot.integration.tests;
-
-public class MultiStageEngineMapTypeClusterIntegrationTest extends MapTypeClusterIntegrationTest {
-
-  @Override
-  protected boolean useMultiStageQueryEngine() {
-    return true;
-  }
-
-  @Override
-  protected int getSelectionDefaultDocCount() {
-    return 1000;
-  }
-
-  @Override
-  public void testMultiValueQueries()
-      throws Exception {
-  }
-}


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