You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@geode.apache.org by GitBox <gi...@apache.org> on 2021/10/18 12:00:40 UTC

[GitHub] [geode] mkevo opened a new pull request #7010: GEODE-9632: fix output for the range query with wildcard character

mkevo opened a new pull request #7010:
URL: https://github.com/apache/geode/pull/7010


   <!-- Thank you for submitting a contribution to Apache Geode. -->
   
   <!-- In order to streamline the review of the contribution we ask you
   to ensure the following steps have been taken: 
   -->
   
   ### For all changes:
   - [x] Is there a JIRA ticket associated with this PR? Is it referenced in the commit message?
   
   - [x] Has your PR been rebased against the latest commit within the target branch (typically `develop`)?
   
   - [x] Is your initial contribution a single, squashed commit?
   
   - [x] Does `gradlew build` run cleanly?
   
   - [ ] Have you written or updated unit tests to verify your changes?
   
   - [ ] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)?
   
   <!-- Note:
   Please ensure that once the PR is submitted, check Concourse for build issues and
   submit an update to your PR as soon as possible. If you need help, please send an
   email to dev@geode.apache.org.
   -->
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

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



[GitHub] [geode] DonalEvans commented on a change in pull request #7010: GEODE-9632: fix output for the range query with wildcard character

Posted by GitBox <gi...@apache.org>.
DonalEvans commented on a change in pull request #7010:
URL: https://github.com/apache/geode/pull/7010#discussion_r736712020



##########
File path: geode-cq/src/distributedTest/java/org/apache/geode/cache/query/dunit/QueryWithRangeIndexDUnitTest.java
##########
@@ -0,0 +1,204 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.geode.cache.query.dunit;
+
+import static org.apache.geode.cache.Region.SEPARATOR;
+import static org.apache.geode.distributed.ConfigurationProperties.HTTP_SERVICE_PORT;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_PORT;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_START;
+import static org.apache.geode.distributed.ConfigurationProperties.LOCATORS;
+import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTCPPorts;
+import static org.apache.geode.internal.lang.SystemPropertyHelper.GEMFIRE_PREFIX;
+import static org.apache.geode.test.awaitility.GeodeAwaitility.await;
+import static org.apache.geode.test.dunit.VM.getVM;
+import static org.apache.geode.test.dunit.VM.getVMId;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.Mockito.mock;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.File;
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.assertj.core.api.Java6Assertions;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+
+import org.apache.geode.DataSerializable;
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.execute.Function;
+import org.apache.geode.cache.execute.FunctionContext;
+import org.apache.geode.cache.execute.FunctionService;
+import org.apache.geode.cache.execute.RegionFunctionContext;
+import org.apache.geode.cache.query.QueryService;
+import org.apache.geode.cache.query.data.Portfolio;
+import org.apache.geode.distributed.LocatorLauncher;
+import org.apache.geode.distributed.ServerLauncher;
+import org.apache.geode.distributed.internal.InternalLocator;
+import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.internal.cache.PartitionedRegion;
+import org.apache.geode.test.dunit.VM;
+import org.apache.geode.test.dunit.rules.DistributedRule;
+import org.apache.geode.test.junit.rules.GfshCommandRule;
+import org.apache.geode.test.junit.rules.serializable.SerializableTemporaryFolder;
+
+public class QueryWithRangeIndexDUnitTest implements Serializable {
+
+  @Rule
+  public transient GfshCommandRule gfsh = new GfshCommandRule();
+
+  @Rule
+  public DistributedRule distributedRule = new DistributedRule();
+
+  @Rule
+  public SerializableTemporaryFolder temporaryFolder = new SerializableTemporaryFolder();
+
+  private String locatorName, serverName;
+
+  private File locatorDir, serverDir;
+
+  private int locatorPort, locatorJmxPort, serverPort;
+
+  private String locators;
+
+  private static final LocatorLauncher DUMMY_LOCATOR = mock(LocatorLauncher.class);

Review comment:
       This constant doesn't seem necessary. What is its purpose?

##########
File path: geode-cq/src/distributedTest/java/org/apache/geode/cache/query/dunit/QueryWithRangeIndexDUnitTest.java
##########
@@ -0,0 +1,204 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.geode.cache.query.dunit;
+
+import static org.apache.geode.cache.Region.SEPARATOR;
+import static org.apache.geode.distributed.ConfigurationProperties.HTTP_SERVICE_PORT;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_PORT;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_START;
+import static org.apache.geode.distributed.ConfigurationProperties.LOCATORS;
+import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTCPPorts;
+import static org.apache.geode.internal.lang.SystemPropertyHelper.GEMFIRE_PREFIX;
+import static org.apache.geode.test.awaitility.GeodeAwaitility.await;
+import static org.apache.geode.test.dunit.VM.getVM;
+import static org.apache.geode.test.dunit.VM.getVMId;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.Mockito.mock;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.File;
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.assertj.core.api.Java6Assertions;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+
+import org.apache.geode.DataSerializable;
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.execute.Function;
+import org.apache.geode.cache.execute.FunctionContext;
+import org.apache.geode.cache.execute.FunctionService;
+import org.apache.geode.cache.execute.RegionFunctionContext;
+import org.apache.geode.cache.query.QueryService;
+import org.apache.geode.cache.query.data.Portfolio;
+import org.apache.geode.distributed.LocatorLauncher;
+import org.apache.geode.distributed.ServerLauncher;
+import org.apache.geode.distributed.internal.InternalLocator;
+import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.internal.cache.PartitionedRegion;
+import org.apache.geode.test.dunit.VM;
+import org.apache.geode.test.dunit.rules.DistributedRule;
+import org.apache.geode.test.junit.rules.GfshCommandRule;
+import org.apache.geode.test.junit.rules.serializable.SerializableTemporaryFolder;
+
+public class QueryWithRangeIndexDUnitTest implements Serializable {
+
+  @Rule
+  public transient GfshCommandRule gfsh = new GfshCommandRule();
+
+  @Rule
+  public DistributedRule distributedRule = new DistributedRule();
+
+  @Rule
+  public SerializableTemporaryFolder temporaryFolder = new SerializableTemporaryFolder();
+
+  private String locatorName, serverName;
+
+  private File locatorDir, serverDir;
+
+  private int locatorPort, locatorJmxPort, serverPort;
+
+  private String locators;
+
+  private static final LocatorLauncher DUMMY_LOCATOR = mock(LocatorLauncher.class);
+
+  private static final AtomicReference<LocatorLauncher> LOCATOR =

Review comment:
       This can be removed and the `startLocator()` method changed to:
   ```
       LocatorLauncher locatorLauncher = new LocatorLauncher.Builder()
           .setMemberName(name)
           .setPort(locatorPort)
           .setWorkingDirectory(workingDirectory.getAbsolutePath())
           .set(JMX_MANAGER, "true")
           .set(JMX_MANAGER_PORT, String.valueOf(jmxPort))
           .set(JMX_MANAGER_START, "true")
           .build();
   
       locatorLauncher.start();
   
       await().untilAsserted(() -> {
         InternalLocator locator = (InternalLocator) locatorLauncher.getLocator();
         assertThat(locator.isSharedConfigurationRunning())
             .as("Locator shared configuration is running on locator" + getVMId())
             .isTrue();
       });
   ```

##########
File path: geode-core/src/main/java/org/apache/geode/cache/query/internal/index/CompactRangeIndex.java
##########
@@ -481,8 +482,13 @@ private void lockedQueryPrivate(Object key, int operator, Collection results,
     int limit = -1;
 
     Boolean applyLimit = (Boolean) context.cacheGet(CompiledValue.CAN_APPLY_LIMIT_AT_INDEX);
+    String indexThresholdSize =
+        System.getProperty(GeodeGlossary.GEMFIRE_PREFIX + "Query.INDEX_THRESHOLD_SIZE");
     if (applyLimit != null && applyLimit) {
       limit = (Integer) context.cacheGet(CompiledValue.RESULT_LIMIT);
+      if (indexThresholdSize != null && limit < Integer.parseInt(indexThresholdSize)) {
+        limit = Integer.parseInt(indexThresholdSize);

Review comment:
       The `lockedQuery()` method in this class also applies a limit. For consistent behaviour, should this change also be applied there? Similarly, other index classes (`HashIndex`, `QueryUtils`, `PrimaryKeyIndex` and `RangeIndex`) use the `CAN_APPLY_LIMIT_AT_INDEX` constant to determine if a limit should be applied, then use the value in `RESULT_LIMIT` to set the limit. This change should be applied in all of those places too, to ensure consistent behaviour.

##########
File path: geode-core/src/main/java/org/apache/geode/cache/query/internal/index/CompactRangeIndex.java
##########
@@ -481,8 +482,13 @@ private void lockedQueryPrivate(Object key, int operator, Collection results,
     int limit = -1;
 
     Boolean applyLimit = (Boolean) context.cacheGet(CompiledValue.CAN_APPLY_LIMIT_AT_INDEX);
+    String indexThresholdSize =
+        System.getProperty(GeodeGlossary.GEMFIRE_PREFIX + "Query.INDEX_THRESHOLD_SIZE");

Review comment:
       Since this value shouldn't be changed at runtime, would it be better to set it when the index is constructed rather than fetching it every time the index is queried? It could also use `Integer.getInteger()` with a default value of 100 (the current default) to avoid having to call `parseInt()` for every query execution.
   
   Also, it seems that with this change, there are now three classes, `CompactRangeIndex`, `AbstractGroupOrRangeJunction`, and `CompiledValue` using this system property. It might make sense to have it defined as a constant in only one place, with a default, and then referenced from there if it needs to be used.

##########
File path: geode-cq/src/distributedTest/java/org/apache/geode/cache/query/dunit/QueryWithRangeIndexDUnitTest.java
##########
@@ -0,0 +1,204 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.geode.cache.query.dunit;
+
+import static org.apache.geode.cache.Region.SEPARATOR;
+import static org.apache.geode.distributed.ConfigurationProperties.HTTP_SERVICE_PORT;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_PORT;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_START;
+import static org.apache.geode.distributed.ConfigurationProperties.LOCATORS;
+import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTCPPorts;
+import static org.apache.geode.internal.lang.SystemPropertyHelper.GEMFIRE_PREFIX;
+import static org.apache.geode.test.awaitility.GeodeAwaitility.await;
+import static org.apache.geode.test.dunit.VM.getVM;
+import static org.apache.geode.test.dunit.VM.getVMId;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.Mockito.mock;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.File;
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.assertj.core.api.Java6Assertions;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+
+import org.apache.geode.DataSerializable;
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.execute.Function;
+import org.apache.geode.cache.execute.FunctionContext;
+import org.apache.geode.cache.execute.FunctionService;
+import org.apache.geode.cache.execute.RegionFunctionContext;
+import org.apache.geode.cache.query.QueryService;
+import org.apache.geode.cache.query.data.Portfolio;
+import org.apache.geode.distributed.LocatorLauncher;
+import org.apache.geode.distributed.ServerLauncher;
+import org.apache.geode.distributed.internal.InternalLocator;
+import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.internal.cache.PartitionedRegion;
+import org.apache.geode.test.dunit.VM;
+import org.apache.geode.test.dunit.rules.DistributedRule;
+import org.apache.geode.test.junit.rules.GfshCommandRule;
+import org.apache.geode.test.junit.rules.serializable.SerializableTemporaryFolder;
+
+public class QueryWithRangeIndexDUnitTest implements Serializable {
+
+  @Rule
+  public transient GfshCommandRule gfsh = new GfshCommandRule();
+
+  @Rule
+  public DistributedRule distributedRule = new DistributedRule();
+
+  @Rule
+  public SerializableTemporaryFolder temporaryFolder = new SerializableTemporaryFolder();
+
+  private String locatorName, serverName;
+
+  private File locatorDir, serverDir;
+
+  private int locatorPort, locatorJmxPort, serverPort;
+
+  private String locators;
+
+  private static final LocatorLauncher DUMMY_LOCATOR = mock(LocatorLauncher.class);
+
+  private static final AtomicReference<LocatorLauncher> LOCATOR =
+      new AtomicReference<>(DUMMY_LOCATOR);
+
+  private static final ServerLauncher DUMMY_SERVER = mock(ServerLauncher.class);
+
+  private static final AtomicReference<ServerLauncher> SERVER =
+      new AtomicReference<>(DUMMY_SERVER);
+
+  private VM locator, server;
+
+  private String regionName;
+
+  @Before
+  public void setUp() throws Exception {
+    locator = getVM(0);
+    server = getVM(1);
+
+    locatorName = "locator";
+    serverName = "server";
+    regionName = "exampleRegion";
+
+    locatorDir = temporaryFolder.newFolder(locatorName);
+    serverDir = temporaryFolder.newFolder(serverName);
+
+    int[] port = getRandomAvailableTCPPorts(3);
+    locatorPort = port[0];
+    locatorJmxPort = port[1];
+    serverPort = port[2];
+
+    locators = "localhost[" + locatorPort + "]";
+
+    locator.invoke(() -> startLocator(locatorName, locatorDir, locatorPort, locatorJmxPort));
+
+    gfsh.connectAndVerify(locatorJmxPort, GfshCommandRule.PortType.jmxManager);
+
+    server.invoke(() -> startServer(serverName, serverDir, serverPort, locators));
+  }
+
+  @Test
+  public void testQueryWithWildcardAndIndexOnAttributeFromHashMap() {
+    gfsh.executeAndAssertThat("create region --name=" + regionName + " --type=PARTITION")
+        .statusIsSuccess();
+
+    server.invoke(() -> {
+      QueryService cacheQS = GemFireCacheImpl.getInstance().getQueryService();

Review comment:
       `GemFireCacheImpl.getInstance()` is deprecated and should not be used. Instead, you can do:
   ```
         Cache cache = SERVER.get().getCache();
         QueryService cacheQS = cache.getQueryService();
   ```
   The `cache` variable can then also be used on line 131 rather than using another call to `GemFireCacheImpl.getInstance()`.

##########
File path: geode-cq/src/distributedTest/java/org/apache/geode/cache/query/dunit/QueryWithRangeIndexDUnitTest.java
##########
@@ -0,0 +1,204 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.geode.cache.query.dunit;
+
+import static org.apache.geode.cache.Region.SEPARATOR;
+import static org.apache.geode.distributed.ConfigurationProperties.HTTP_SERVICE_PORT;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_PORT;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_START;
+import static org.apache.geode.distributed.ConfigurationProperties.LOCATORS;
+import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTCPPorts;
+import static org.apache.geode.internal.lang.SystemPropertyHelper.GEMFIRE_PREFIX;
+import static org.apache.geode.test.awaitility.GeodeAwaitility.await;
+import static org.apache.geode.test.dunit.VM.getVM;
+import static org.apache.geode.test.dunit.VM.getVMId;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.Mockito.mock;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.File;
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.assertj.core.api.Java6Assertions;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+
+import org.apache.geode.DataSerializable;
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.execute.Function;
+import org.apache.geode.cache.execute.FunctionContext;
+import org.apache.geode.cache.execute.FunctionService;
+import org.apache.geode.cache.execute.RegionFunctionContext;
+import org.apache.geode.cache.query.QueryService;
+import org.apache.geode.cache.query.data.Portfolio;
+import org.apache.geode.distributed.LocatorLauncher;
+import org.apache.geode.distributed.ServerLauncher;
+import org.apache.geode.distributed.internal.InternalLocator;
+import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.internal.cache.PartitionedRegion;
+import org.apache.geode.test.dunit.VM;
+import org.apache.geode.test.dunit.rules.DistributedRule;
+import org.apache.geode.test.junit.rules.GfshCommandRule;
+import org.apache.geode.test.junit.rules.serializable.SerializableTemporaryFolder;
+
+public class QueryWithRangeIndexDUnitTest implements Serializable {
+
+  @Rule
+  public transient GfshCommandRule gfsh = new GfshCommandRule();
+
+  @Rule
+  public DistributedRule distributedRule = new DistributedRule();
+
+  @Rule
+  public SerializableTemporaryFolder temporaryFolder = new SerializableTemporaryFolder();
+
+  private String locatorName, serverName;
+
+  private File locatorDir, serverDir;
+
+  private int locatorPort, locatorJmxPort, serverPort;
+
+  private String locators;
+
+  private static final LocatorLauncher DUMMY_LOCATOR = mock(LocatorLauncher.class);
+
+  private static final AtomicReference<LocatorLauncher> LOCATOR =
+      new AtomicReference<>(DUMMY_LOCATOR);
+
+  private static final ServerLauncher DUMMY_SERVER = mock(ServerLauncher.class);

Review comment:
       This constant doesn't seem necessary. What is its purpose?

##########
File path: geode-cq/src/distributedTest/java/org/apache/geode/cache/query/dunit/QueryWithRangeIndexDUnitTest.java
##########
@@ -0,0 +1,204 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.geode.cache.query.dunit;
+
+import static org.apache.geode.cache.Region.SEPARATOR;
+import static org.apache.geode.distributed.ConfigurationProperties.HTTP_SERVICE_PORT;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_PORT;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_START;
+import static org.apache.geode.distributed.ConfigurationProperties.LOCATORS;
+import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTCPPorts;
+import static org.apache.geode.internal.lang.SystemPropertyHelper.GEMFIRE_PREFIX;
+import static org.apache.geode.test.awaitility.GeodeAwaitility.await;
+import static org.apache.geode.test.dunit.VM.getVM;
+import static org.apache.geode.test.dunit.VM.getVMId;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.Mockito.mock;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.File;
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.assertj.core.api.Java6Assertions;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+
+import org.apache.geode.DataSerializable;
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.execute.Function;
+import org.apache.geode.cache.execute.FunctionContext;
+import org.apache.geode.cache.execute.FunctionService;
+import org.apache.geode.cache.execute.RegionFunctionContext;
+import org.apache.geode.cache.query.QueryService;
+import org.apache.geode.cache.query.data.Portfolio;
+import org.apache.geode.distributed.LocatorLauncher;
+import org.apache.geode.distributed.ServerLauncher;
+import org.apache.geode.distributed.internal.InternalLocator;
+import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.internal.cache.PartitionedRegion;
+import org.apache.geode.test.dunit.VM;
+import org.apache.geode.test.dunit.rules.DistributedRule;
+import org.apache.geode.test.junit.rules.GfshCommandRule;
+import org.apache.geode.test.junit.rules.serializable.SerializableTemporaryFolder;
+
+public class QueryWithRangeIndexDUnitTest implements Serializable {
+
+  @Rule
+  public transient GfshCommandRule gfsh = new GfshCommandRule();
+
+  @Rule
+  public DistributedRule distributedRule = new DistributedRule();
+
+  @Rule
+  public SerializableTemporaryFolder temporaryFolder = new SerializableTemporaryFolder();
+
+  private String locatorName, serverName;
+
+  private File locatorDir, serverDir;
+
+  private int locatorPort, locatorJmxPort, serverPort;
+
+  private String locators;
+
+  private static final LocatorLauncher DUMMY_LOCATOR = mock(LocatorLauncher.class);
+
+  private static final AtomicReference<LocatorLauncher> LOCATOR =
+      new AtomicReference<>(DUMMY_LOCATOR);
+
+  private static final ServerLauncher DUMMY_SERVER = mock(ServerLauncher.class);
+
+  private static final AtomicReference<ServerLauncher> SERVER =
+      new AtomicReference<>(DUMMY_SERVER);
+
+  private VM locator, server;
+
+  private String regionName;
+
+  @Before
+  public void setUp() throws Exception {
+    locator = getVM(0);
+    server = getVM(1);
+
+    locatorName = "locator";
+    serverName = "server";
+    regionName = "exampleRegion";
+
+    locatorDir = temporaryFolder.newFolder(locatorName);
+    serverDir = temporaryFolder.newFolder(serverName);
+
+    int[] port = getRandomAvailableTCPPorts(3);
+    locatorPort = port[0];
+    locatorJmxPort = port[1];
+    serverPort = port[2];
+
+    locators = "localhost[" + locatorPort + "]";
+
+    locator.invoke(() -> startLocator(locatorName, locatorDir, locatorPort, locatorJmxPort));
+
+    gfsh.connectAndVerify(locatorJmxPort, GfshCommandRule.PortType.jmxManager);
+
+    server.invoke(() -> startServer(serverName, serverDir, serverPort, locators));
+  }
+
+  @Test
+  public void testQueryWithWildcardAndIndexOnAttributeFromHashMap() {
+    gfsh.executeAndAssertThat("create region --name=" + regionName + " --type=PARTITION")
+        .statusIsSuccess();
+
+    server.invoke(() -> {
+      QueryService cacheQS = GemFireCacheImpl.getInstance().getQueryService();
+      cacheQS.createIndex("IdIndex", "value.positions['SUN']",
+          SEPARATOR + regionName + ".entrySet");
+      Region<Integer, Portfolio> region =
+          GemFireCacheImpl.getInstance().getRegion(regionName);
+      FunctionService.onRegion(region).execute(new MyFunction());
+      await().untilAsserted(() -> assertThat(region.size()).isEqualTo(10000));
+    });
+
+    String query = "query --query=\"<trace> select e.key, e.value from " +
+        SEPARATOR + regionName + ".entrySet e where e.value.positions['SUN'] like 'somethin%'\"";
+
+    String cmdResult = String.valueOf(gfsh.executeAndAssertThat(query).getResultModel());
+    assertThat(cmdResult).contains("\"Rows\":\"1\"");
+    assertThat(cmdResult).contains("indexesUsed(1):IdIndex(Results: 10000)");
+  }
+
+  private static void startLocator(String name, File workingDirectory, int locatorPort,
+      int jmxPort) {
+    LOCATOR.set(new LocatorLauncher.Builder()
+        .setMemberName(name)
+        .setPort(locatorPort)
+        .setWorkingDirectory(workingDirectory.getAbsolutePath())
+        .set(JMX_MANAGER, "true")
+        .set(JMX_MANAGER_PORT, String.valueOf(jmxPort))
+        .set(JMX_MANAGER_START, "true")
+        .build());
+
+    LOCATOR.get().start();
+
+    await().untilAsserted(() -> {
+      InternalLocator locator = (InternalLocator) LOCATOR.get().getLocator();
+      assertThat(locator.isSharedConfigurationRunning())
+          .as("Locator shared configuration is running on locator" + getVMId())
+          .isTrue();
+    });
+  }
+
+  private static void startServer(String name, File workingDirectory, int serverPort,
+      String locators) {
+    System.setProperty(GEMFIRE_PREFIX + "Query.INDEX_THRESHOLD_SIZE", "10000");
+    SERVER.set(new ServerLauncher.Builder()
+        .setDeletePidFileOnStop(Boolean.TRUE)
+        .setMemberName(name)
+        .setServerPort(serverPort)
+        .setWorkingDirectory(workingDirectory.getAbsolutePath())
+        .set(HTTP_SERVICE_PORT, "0")
+        .set(LOCATORS, locators)
+        .build());
+
+    SERVER.get().start();
+  }
+
+  public static class MyFunction implements Function, DataSerializable {
+    @Override
+    public void execute(FunctionContext context) {
+      Java6Assertions.assertThat(context).isInstanceOf(RegionFunctionContext.class);
+      PartitionedRegion region = (PartitionedRegion) ((RegionFunctionContext) context).getDataSet();

Review comment:
       This can be simplified to:
   ```
   Region<Integer, Portfolio> region = context.getCache().getRegion(regionName);
   ```
   if `regionName` is made a constant.

##########
File path: geode-cq/src/distributedTest/java/org/apache/geode/cache/query/dunit/QueryWithRangeIndexDUnitTest.java
##########
@@ -0,0 +1,204 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.geode.cache.query.dunit;
+
+import static org.apache.geode.cache.Region.SEPARATOR;
+import static org.apache.geode.distributed.ConfigurationProperties.HTTP_SERVICE_PORT;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_PORT;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_START;
+import static org.apache.geode.distributed.ConfigurationProperties.LOCATORS;
+import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTCPPorts;
+import static org.apache.geode.internal.lang.SystemPropertyHelper.GEMFIRE_PREFIX;
+import static org.apache.geode.test.awaitility.GeodeAwaitility.await;
+import static org.apache.geode.test.dunit.VM.getVM;
+import static org.apache.geode.test.dunit.VM.getVMId;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.Mockito.mock;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.File;
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.assertj.core.api.Java6Assertions;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+
+import org.apache.geode.DataSerializable;
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.execute.Function;
+import org.apache.geode.cache.execute.FunctionContext;
+import org.apache.geode.cache.execute.FunctionService;
+import org.apache.geode.cache.execute.RegionFunctionContext;
+import org.apache.geode.cache.query.QueryService;
+import org.apache.geode.cache.query.data.Portfolio;
+import org.apache.geode.distributed.LocatorLauncher;
+import org.apache.geode.distributed.ServerLauncher;
+import org.apache.geode.distributed.internal.InternalLocator;
+import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.internal.cache.PartitionedRegion;
+import org.apache.geode.test.dunit.VM;
+import org.apache.geode.test.dunit.rules.DistributedRule;
+import org.apache.geode.test.junit.rules.GfshCommandRule;
+import org.apache.geode.test.junit.rules.serializable.SerializableTemporaryFolder;
+
+public class QueryWithRangeIndexDUnitTest implements Serializable {
+
+  @Rule
+  public transient GfshCommandRule gfsh = new GfshCommandRule();
+
+  @Rule
+  public DistributedRule distributedRule = new DistributedRule();
+
+  @Rule
+  public SerializableTemporaryFolder temporaryFolder = new SerializableTemporaryFolder();
+
+  private String locatorName, serverName;

Review comment:
       These Strings are never modified, so they can be made constants, similarly for `regionName`.

##########
File path: geode-cq/src/distributedTest/java/org/apache/geode/cache/query/dunit/QueryWithRangeIndexDUnitTest.java
##########
@@ -0,0 +1,204 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.geode.cache.query.dunit;
+
+import static org.apache.geode.cache.Region.SEPARATOR;
+import static org.apache.geode.distributed.ConfigurationProperties.HTTP_SERVICE_PORT;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_PORT;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_START;
+import static org.apache.geode.distributed.ConfigurationProperties.LOCATORS;
+import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTCPPorts;
+import static org.apache.geode.internal.lang.SystemPropertyHelper.GEMFIRE_PREFIX;
+import static org.apache.geode.test.awaitility.GeodeAwaitility.await;
+import static org.apache.geode.test.dunit.VM.getVM;
+import static org.apache.geode.test.dunit.VM.getVMId;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.Mockito.mock;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.File;
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.assertj.core.api.Java6Assertions;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+
+import org.apache.geode.DataSerializable;
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.execute.Function;
+import org.apache.geode.cache.execute.FunctionContext;
+import org.apache.geode.cache.execute.FunctionService;
+import org.apache.geode.cache.execute.RegionFunctionContext;
+import org.apache.geode.cache.query.QueryService;
+import org.apache.geode.cache.query.data.Portfolio;
+import org.apache.geode.distributed.LocatorLauncher;
+import org.apache.geode.distributed.ServerLauncher;
+import org.apache.geode.distributed.internal.InternalLocator;
+import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.internal.cache.PartitionedRegion;
+import org.apache.geode.test.dunit.VM;
+import org.apache.geode.test.dunit.rules.DistributedRule;
+import org.apache.geode.test.junit.rules.GfshCommandRule;
+import org.apache.geode.test.junit.rules.serializable.SerializableTemporaryFolder;
+
+public class QueryWithRangeIndexDUnitTest implements Serializable {
+
+  @Rule
+  public transient GfshCommandRule gfsh = new GfshCommandRule();
+
+  @Rule
+  public DistributedRule distributedRule = new DistributedRule();
+
+  @Rule
+  public SerializableTemporaryFolder temporaryFolder = new SerializableTemporaryFolder();
+
+  private String locatorName, serverName;
+
+  private File locatorDir, serverDir;
+
+  private int locatorPort, locatorJmxPort, serverPort;
+
+  private String locators;
+
+  private static final LocatorLauncher DUMMY_LOCATOR = mock(LocatorLauncher.class);
+
+  private static final AtomicReference<LocatorLauncher> LOCATOR =
+      new AtomicReference<>(DUMMY_LOCATOR);
+
+  private static final ServerLauncher DUMMY_SERVER = mock(ServerLauncher.class);
+
+  private static final AtomicReference<ServerLauncher> SERVER =
+      new AtomicReference<>(DUMMY_SERVER);
+
+  private VM locator, server;
+
+  private String regionName;
+
+  @Before
+  public void setUp() throws Exception {
+    locator = getVM(0);
+    server = getVM(1);
+
+    locatorName = "locator";
+    serverName = "server";
+    regionName = "exampleRegion";
+
+    locatorDir = temporaryFolder.newFolder(locatorName);
+    serverDir = temporaryFolder.newFolder(serverName);
+
+    int[] port = getRandomAvailableTCPPorts(3);
+    locatorPort = port[0];
+    locatorJmxPort = port[1];
+    serverPort = port[2];
+
+    locators = "localhost[" + locatorPort + "]";
+
+    locator.invoke(() -> startLocator(locatorName, locatorDir, locatorPort, locatorJmxPort));
+
+    gfsh.connectAndVerify(locatorJmxPort, GfshCommandRule.PortType.jmxManager);
+
+    server.invoke(() -> startServer(serverName, serverDir, serverPort, locators));
+  }
+
+  @Test
+  public void testQueryWithWildcardAndIndexOnAttributeFromHashMap() {
+    gfsh.executeAndAssertThat("create region --name=" + regionName + " --type=PARTITION")
+        .statusIsSuccess();
+
+    server.invoke(() -> {
+      QueryService cacheQS = GemFireCacheImpl.getInstance().getQueryService();
+      cacheQS.createIndex("IdIndex", "value.positions['SUN']",
+          SEPARATOR + regionName + ".entrySet");
+      Region<Integer, Portfolio> region =
+          GemFireCacheImpl.getInstance().getRegion(regionName);
+      FunctionService.onRegion(region).execute(new MyFunction());
+      await().untilAsserted(() -> assertThat(region.size()).isEqualTo(10000));
+    });
+
+    String query = "query --query=\"<trace> select e.key, e.value from " +
+        SEPARATOR + regionName + ".entrySet e where e.value.positions['SUN'] like 'somethin%'\"";
+
+    String cmdResult = String.valueOf(gfsh.executeAndAssertThat(query).getResultModel());
+    assertThat(cmdResult).contains("\"Rows\":\"1\"");
+    assertThat(cmdResult).contains("indexesUsed(1):IdIndex(Results: 10000)");
+  }
+
+  private static void startLocator(String name, File workingDirectory, int locatorPort,
+      int jmxPort) {
+    LOCATOR.set(new LocatorLauncher.Builder()
+        .setMemberName(name)
+        .setPort(locatorPort)
+        .setWorkingDirectory(workingDirectory.getAbsolutePath())
+        .set(JMX_MANAGER, "true")
+        .set(JMX_MANAGER_PORT, String.valueOf(jmxPort))
+        .set(JMX_MANAGER_START, "true")
+        .build());
+
+    LOCATOR.get().start();
+
+    await().untilAsserted(() -> {
+      InternalLocator locator = (InternalLocator) LOCATOR.get().getLocator();
+      assertThat(locator.isSharedConfigurationRunning())
+          .as("Locator shared configuration is running on locator" + getVMId())
+          .isTrue();
+    });
+  }
+
+  private static void startServer(String name, File workingDirectory, int serverPort,
+      String locators) {
+    System.setProperty(GEMFIRE_PREFIX + "Query.INDEX_THRESHOLD_SIZE", "10000");
+    SERVER.set(new ServerLauncher.Builder()
+        .setDeletePidFileOnStop(Boolean.TRUE)
+        .setMemberName(name)
+        .setServerPort(serverPort)
+        .setWorkingDirectory(workingDirectory.getAbsolutePath())
+        .set(HTTP_SERVICE_PORT, "0")
+        .set(LOCATORS, locators)
+        .build());
+
+    SERVER.get().start();
+  }
+
+  public static class MyFunction implements Function, DataSerializable {
+    @Override
+    public void execute(FunctionContext context) {

Review comment:
       Warnings here can be fixed by using `Function<Void>` and `FunctionContext<Void>`. Alternately, these puts could just be done inline in the test inside the invocation on the server instead of using function execution, since there doesn't seem to be a reason for using it here.

##########
File path: geode-cq/src/distributedTest/java/org/apache/geode/cache/query/dunit/QueryWithRangeIndexDUnitTest.java
##########
@@ -0,0 +1,204 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.geode.cache.query.dunit;
+
+import static org.apache.geode.cache.Region.SEPARATOR;
+import static org.apache.geode.distributed.ConfigurationProperties.HTTP_SERVICE_PORT;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_PORT;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_START;
+import static org.apache.geode.distributed.ConfigurationProperties.LOCATORS;
+import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTCPPorts;
+import static org.apache.geode.internal.lang.SystemPropertyHelper.GEMFIRE_PREFIX;
+import static org.apache.geode.test.awaitility.GeodeAwaitility.await;
+import static org.apache.geode.test.dunit.VM.getVM;
+import static org.apache.geode.test.dunit.VM.getVMId;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.Mockito.mock;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.File;
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.assertj.core.api.Java6Assertions;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+
+import org.apache.geode.DataSerializable;
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.execute.Function;
+import org.apache.geode.cache.execute.FunctionContext;
+import org.apache.geode.cache.execute.FunctionService;
+import org.apache.geode.cache.execute.RegionFunctionContext;
+import org.apache.geode.cache.query.QueryService;
+import org.apache.geode.cache.query.data.Portfolio;
+import org.apache.geode.distributed.LocatorLauncher;
+import org.apache.geode.distributed.ServerLauncher;
+import org.apache.geode.distributed.internal.InternalLocator;
+import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.internal.cache.PartitionedRegion;
+import org.apache.geode.test.dunit.VM;
+import org.apache.geode.test.dunit.rules.DistributedRule;
+import org.apache.geode.test.junit.rules.GfshCommandRule;
+import org.apache.geode.test.junit.rules.serializable.SerializableTemporaryFolder;
+
+public class QueryWithRangeIndexDUnitTest implements Serializable {
+
+  @Rule
+  public transient GfshCommandRule gfsh = new GfshCommandRule();
+
+  @Rule
+  public DistributedRule distributedRule = new DistributedRule();
+
+  @Rule
+  public SerializableTemporaryFolder temporaryFolder = new SerializableTemporaryFolder();
+
+  private String locatorName, serverName;
+
+  private File locatorDir, serverDir;
+
+  private int locatorPort, locatorJmxPort, serverPort;

Review comment:
       The [style guide used by Geode](https://google.github.io/styleguide/javaguide.html#s4.8.2-variable-declarations) requires that only one variable is defined per line. These should be broken up onto separate lines.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

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



[GitHub] [geode] DonalEvans commented on pull request #7010: GEODE-9632: fix output for the range query with wildcard character

Posted by GitBox <gi...@apache.org>.
DonalEvans commented on pull request #7010:
URL: https://github.com/apache/geode/pull/7010#issuecomment-952147519


   In addition to the changes requested, the name of this PR should be changed to describe what's actually being changed, not just the intended effect. Something like "Allow INDEX_THRESHOLD_SIZE System property to override CompiledValue.RESULT_LIMIT"


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

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



[GitHub] [geode] mkevo merged pull request #7010: GEODE-9632: Allow INDEX_THRESHOLD_SIZE System property to override CompiledValue.RESULT_LIMIT

Posted by GitBox <gi...@apache.org>.
mkevo merged pull request #7010:
URL: https://github.com/apache/geode/pull/7010


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

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



[GitHub] [geode] mkevo commented on pull request #7010: GEODE-9632: fix output for the range query with wildcard character

Posted by GitBox <gi...@apache.org>.
mkevo commented on pull request #7010:
URL: https://github.com/apache/geode/pull/7010#issuecomment-951662578


   The test failing is random, and doesn't have anything related to existing changes. I run it multiple times and it passed.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

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



[GitHub] [geode] mkevo commented on pull request #7010: GEODE-9632: Allow INDEX_THRESHOLD_SIZE System property to override CompiledValue.RESULT_LIMIT

Posted by GitBox <gi...@apache.org>.
mkevo commented on pull request #7010:
URL: https://github.com/apache/geode/pull/7010#issuecomment-963041221


   The problem in failing tests is that with a fix, the limit added in the query will be applied only on results, not on index results. Size of indexResults is defined by INDEX_THRESHOLD_SIZE. A new test case(_testLimitIsAppliedOnlyOnQueryResults_ in QueryWithRangeIndexDUnitTest.java) is added to verify it. 
   Not sure what to do with failing tests, as they have assertions to check if limitAppliedAtIndex, and it will not apply as it checks INDEX_THRESHOLD_SIZE for limit at index.
   Also, this can be fixed by the first proposal with fetching INDEX_THRESHOLD_SIZE every time the index is queried, but think that it is not the best solution as it will not take INDEX_THRESHOLD_SIZE into a count if it is not specified in the server startup command. This can lead to not collecting results in indexResults and the result of the query can be wrong.
   
   Also, the PR name should be something like: _GEODE-9632: Add option to apply INDEX_THRESHOLD_SIZE on index results_, as it will not overwrite RESULT_LIMIT for printing results, just for collection of index results.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

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



[GitHub] [geode] DonalEvans commented on pull request #7010: GEODE-9632: Allow INDEX_THRESHOLD_SIZE System property to override CompiledValue.RESULT_LIMIT

Posted by GitBox <gi...@apache.org>.
DonalEvans commented on pull request #7010:
URL: https://github.com/apache/geode/pull/7010#issuecomment-966669272


   The reason that the tests are currently failing is that there are only 50 entries in the region but the default value for the index threshold limit is 100, so that limit is not being surpassed and so the `limitAppliedAtIndex` flag is not being set to true. Increasing the number of objects in the region to a value above 100 results in the tests passing. I think that this fix is fine, since the previous assertion was based on the incorrect behaviour which this PR is fixing.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

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



[GitHub] [geode] mkevo commented on pull request #7010: GEODE-9632: fix output for the range query with wildcard character

Posted by GitBox <gi...@apache.org>.
mkevo commented on pull request #7010:
URL: https://github.com/apache/geode/pull/7010#issuecomment-951585734


   @agingade, @nabarunnag, @DonalEvans, Can you please review it? 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

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