You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by GitBox <gi...@apache.org> on 2022/02/22 12:31:31 UTC

[GitHub] [ignite] timoninmaxim opened a new pull request #9846: IGNITE-16604: Fix GeoSpatial indexes for client nodes

timoninmaxim opened a new pull request #9846:
URL: https://github.com/apache/ignite/pull/9846


   Client nodes incorrectly handle geo-spatial indexes, and fail on trying to create index. This patch provides a mock for geo-spatial indexes on client nodes by analogue with ClientInlineIndex for sorted indexes. It enables creation of indexes on Ignite client nodes.


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

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

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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9846: IGNITE-16604: Fix GeoSpatial indexes for client nodes

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9846:
URL: https://github.com/apache/ignite/pull/9846#discussion_r830866510



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/index/client/ClientInlineIndex.java
##########
@@ -148,7 +148,7 @@ public ClientInlineIndex(String idxName, int inlineSize) {
     /**
      * @return Exception about unsupported operation.
      */
-    private static IgniteException unsupported() {
+    public static IgniteException unsupported() {

Review comment:
       I'm not sure that it's OK to move this method to U class. This method is expected to be used only by client indexes. IMHO, the better option is an additional layer like `AbstractClientIndex` with this method. 




-- 
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@ignite.apache.org

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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9846: IGNITE-16604: Fix GeoSpatial indexes for client nodes

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9846:
URL: https://github.com/apache/ignite/pull/9846#discussion_r830909813



##########
File path: modules/geospatial/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GeoSpatialUtils.java
##########
@@ -46,27 +46,53 @@
     }
 
     /** */
-    public static GridH2SpatialIndex createIndex(GridH2Table tbl, String idxName, List<IndexColumn> cols) {
+    public static GridH2IndexBase createIndex(GridH2Table tbl, String idxName, List<IndexColumn> cols) {
         try {
             IndexName name = new IndexName(tbl.cacheName(), tbl.getSchema().getName(), tbl.getName(), idxName);
 
             LinkedHashMap<String, IndexKeyDefinition> keyDefs = new QueryIndexKeyDefinitionProvider(tbl, cols).keyDefinitions();
 
-            List<InlineIndexKeyType> idxKeyTypes = InlineIndexKeyTypeRegistry.types(keyDefs.values(), DUMMY_SETTINGS);
+            if (tbl.cacheInfo().affinityNode())

Review comment:
       No, node filters are checked before the index creation procedure.




-- 
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@ignite.apache.org

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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9846: IGNITE-16604: Fix GeoSpatial indexes for client nodes

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9846:
URL: https://github.com/apache/ignite/pull/9846#discussion_r830896344



##########
File path: modules/geospatial/src/test/java/org/apache/ignite/internal/processors/query/h2/H2IndexingAbstractGeoSelfTest.java
##########
@@ -629,11 +663,11 @@ public Enemy(int campId, String name) {
     protected static class EnemyCamp implements Serializable {
         /** */
         @QuerySqlField(index = true)
-        Geometry coords;
+        final Geometry coords;

Review comment:
       not, it used in `checkDistributedQuery()` by direct reference.




-- 
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@ignite.apache.org

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



[GitHub] [ignite] anton-vinogradov commented on a change in pull request #9846: IGNITE-16604: Fix GeoSpatial indexes for client nodes

Posted by GitBox <gi...@apache.org>.
anton-vinogradov commented on a change in pull request #9846:
URL: https://github.com/apache/ignite/pull/9846#discussion_r830953262



##########
File path: modules/geospatial/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GeoSpatialClientIndexDefinition.java
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.ignite.internal.processors.query.h2.opt;
+
+import java.util.LinkedHashMap;
+import java.util.List;
+import org.apache.ignite.internal.cache.query.index.IndexDefinition;
+import org.apache.ignite.internal.cache.query.index.IndexName;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyDefinition;
+import org.h2.table.IndexColumn;
+
+/**
+ * Definition of Geo-Spatial client index.
+ */
+public class GeoSpatialClientIndexDefinition implements IndexDefinition {
+    /** */
+    private final LinkedHashMap<String, IndexKeyDefinition> keyDefs;
+
+    /** */
+    private final IndexName idxName;
+
+    /** */
+    private final GridH2Table tbl;
+
+    /** */
+    private final List<IndexColumn> cols;

Review comment:
       Just a recheck proposal.




-- 
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@ignite.apache.org

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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9846: IGNITE-16604: Fix GeoSpatial indexes for client nodes

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9846:
URL: https://github.com/apache/ignite/pull/9846#discussion_r830866510



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/index/client/ClientInlineIndex.java
##########
@@ -148,7 +148,7 @@ public ClientInlineIndex(String idxName, int inlineSize) {
     /**
      * @return Exception about unsupported operation.
      */
-    private static IgniteException unsupported() {
+    public static IgniteException unsupported() {

Review comment:
       I'm not sure that it's OK to move this method to U class. This method is expected to be used only by client nodes. IMHO, the better option is an additional layer like `AbstractClientIndex` with this method. 




-- 
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@ignite.apache.org

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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9846: IGNITE-16604: Fix GeoSpatial indexes for client nodes

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9846:
URL: https://github.com/apache/ignite/pull/9846#discussion_r830877955



##########
File path: modules/geospatial/src/test/java/org/apache/ignite/internal/processors/query/h2/H2IndexingAbstractGeoSelfTest.java
##########
@@ -76,6 +80,28 @@
     /** Segmented index flag. */
     private final boolean segmented;
 
+    /** Node that creates a Geo-Spatial index. */
+    @Parameterized.Parameter
+    public String crtNode;
+
+    /** Node that stars a query over a Geo-Spatial index. */
+    @Parameterized.Parameter(1)
+    public String qryNode;
+
+    /** */
+    private IgniteEx cln;
+
+    /** */
+    @Parameterized.Parameters(name = "crtNode={0}, qryNode={1}")
+    public static List<Object[]> parameters() {
+        return F.asList(
+            new Object[] { "CRD", "CRD" },
+            new Object[] { "CRD", "CLN" },
+            new Object[] { "CLN", "CRD" },
+            new Object[] { "CLN", "CLN" }

Review comment:
       There is even better solution - use cartesian product of array `{ "CLN", "CRD" }` on itself.




-- 
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@ignite.apache.org

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



[GitHub] [ignite] anton-vinogradov commented on a change in pull request #9846: IGNITE-16604: Fix GeoSpatial indexes for client nodes

Posted by GitBox <gi...@apache.org>.
anton-vinogradov commented on a change in pull request #9846:
URL: https://github.com/apache/ignite/pull/9846#discussion_r830935915



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/index/client/ClientInlineIndex.java
##########
@@ -148,7 +148,7 @@ public ClientInlineIndex(String idxName, int inlineSize) {
     /**
      * @return Exception about unsupported operation.
      */
-    private static IgniteException unsupported() {
+    public static IgniteException unsupported() {

Review comment:
       Any ancestor also is good choice




-- 
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@ignite.apache.org

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



[GitHub] [ignite] anton-vinogradov commented on a change in pull request #9846: IGNITE-16604: Fix GeoSpatial indexes for client nodes

Posted by GitBox <gi...@apache.org>.
anton-vinogradov commented on a change in pull request #9846:
URL: https://github.com/apache/ignite/pull/9846#discussion_r824596209



##########
File path: modules/geospatial/src/test/java/org/apache/ignite/internal/processors/query/h2/H2IndexingAbstractGeoSelfTest.java
##########
@@ -122,11 +159,12 @@ protected H2IndexingAbstractGeoSelfTest(boolean segmented) {
      * @return Cache.
      * @throws Exception If failed.
      */
-    @SuppressWarnings("unchecked")
     protected <K, V> IgniteCache<K, V> createCache(String name, boolean partitioned, Class<?> keyCls, Class<?> valCls,
         boolean dynamicIdx) throws Exception {
         CacheConfiguration<K, V> ccfg = cacheConfig(name, partitioned, keyCls, valCls);
 
+        IgniteEx crtIgn = node(crtNode);

Review comment:
       use ternary operation here to chose between `cln` and `srv`




-- 
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@ignite.apache.org

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



[GitHub] [ignite] anton-vinogradov commented on a change in pull request #9846: IGNITE-16604: Fix GeoSpatial indexes for client nodes

Posted by GitBox <gi...@apache.org>.
anton-vinogradov commented on a change in pull request #9846:
URL: https://github.com/apache/ignite/pull/9846#discussion_r816848138



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/index/client/ClientInlineIndex.java
##########
@@ -148,7 +148,7 @@ public ClientInlineIndex(String idxName, int inlineSize) {
     /**
      * @return Exception about unsupported operation.
      */
-    private static IgniteException unsupported() {
+    public static IgniteException unsupported() {

Review comment:
       Should this be relocated to some U method and used everywhere?
   I mean that all such throws (eg. at GriH2SpatialBaseIndex) should be replaced by util methods.

##########
File path: modules/geospatial/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GeoSpatialClientIndex.java
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.ignite.internal.processors.query.h2.opt;
+
+import java.util.List;
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.cache.query.index.AbstractIndex;
+import org.apache.ignite.internal.cache.query.index.Index;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexRow;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.util.lang.GridCursor;
+import org.h2.table.IndexColumn;
+import org.h2.table.TableFilter;
+import org.jetbrains.annotations.Nullable;
+import org.locationtech.jts.geom.Geometry;
+
+import static org.apache.ignite.internal.processors.query.h2.index.client.ClientInlineIndex.unsupported;
+
+/**
+ * Mock for client nodes to support Geo-Spatial indexes.
+ */
+public class GeoSpatialClientIndex extends AbstractIndex implements GeoSpatialIndex {
+    /** Index unique ID. */
+    private final UUID id = UUID.randomUUID();
+
+    /** */
+    private final String name;
+
+    /** */
+    private final GridH2Table tbl;
+
+    /** */
+    private final List<IndexColumn> cols;
+
+    /**
+     * @param def Index definition.
+     */
+    public GeoSpatialClientIndex(GeoSpatialClientIndexDefinition def) {
+        name = def.idxName().idxName();
+        tbl = def.tbl();
+        cols = def.cols();
+    }
+
+    /** {@inheritDoc} */
+    @Override public UUID id() {
+        return id;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String name() {
+        return name;
+    }
+
+    /** */
+    public GridH2Table tbl() {
+        return tbl;
+    }
+
+    /** */
+    public List<IndexColumn> cols() {
+        return cols;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean canHandle(CacheDataRow row) throws IgniteCheckedException {

Review comment:
       The throw is not necessary here.

##########
File path: modules/geospatial/src/test/java/org/apache/ignite/internal/processors/query/h2/H2IndexingAbstractGeoSelfTest.java
##########
@@ -76,6 +80,28 @@
     /** Segmented index flag. */
     private final boolean segmented;
 
+    /** Node that creates a Geo-Spatial index. */
+    @Parameterized.Parameter
+    public String crtNode;
+
+    /** Node that stars a query over a Geo-Spatial index. */
+    @Parameterized.Parameter(1)
+    public String qryNode;
+
+    /** */
+    private IgniteEx cln;
+
+    /** */
+    @Parameterized.Parameters(name = "crtNode={0}, qryNode={1}")
+    public static List<Object[]> parameters() {
+        return F.asList(
+            new Object[] { "CRD", "CRD" },
+            new Object[] { "CRD", "CLN" },
+            new Object[] { "CLN", "CRD" },
+            new Object[] { "CLN", "CLN" }

Review comment:
       Please use double-for to avoid possible typos (avoid possible duplication with missing some case)

##########
File path: modules/geospatial/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GeoSpatialClientIndex.java
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.ignite.internal.processors.query.h2.opt;
+
+import java.util.List;
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.cache.query.index.AbstractIndex;
+import org.apache.ignite.internal.cache.query.index.Index;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexRow;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.util.lang.GridCursor;
+import org.h2.table.IndexColumn;
+import org.h2.table.TableFilter;
+import org.jetbrains.annotations.Nullable;
+import org.locationtech.jts.geom.Geometry;
+
+import static org.apache.ignite.internal.processors.query.h2.index.client.ClientInlineIndex.unsupported;
+
+/**
+ * Mock for client nodes to support Geo-Spatial indexes.
+ */
+public class GeoSpatialClientIndex extends AbstractIndex implements GeoSpatialIndex {
+    /** Index unique ID. */
+    private final UUID id = UUID.randomUUID();
+
+    /** */
+    private final String name;
+
+    /** */
+    private final GridH2Table tbl;
+
+    /** */
+    private final List<IndexColumn> cols;

Review comment:
       Could we keep GeoSpatialClientIndexDefinition instead?

##########
File path: modules/geospatial/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GeoSpatialUtils.java
##########
@@ -46,27 +46,53 @@
     }
 
     /** */
-    public static GridH2SpatialIndex createIndex(GridH2Table tbl, String idxName, List<IndexColumn> cols) {
+    public static GridH2IndexBase createIndex(GridH2Table tbl, String idxName, List<IndexColumn> cols) {
         try {
             IndexName name = new IndexName(tbl.cacheName(), tbl.getSchema().getName(), tbl.getName(), idxName);
 
             LinkedHashMap<String, IndexKeyDefinition> keyDefs = new QueryIndexKeyDefinitionProvider(tbl, cols).keyDefinitions();
 
-            List<InlineIndexKeyType> idxKeyTypes = InlineIndexKeyTypeRegistry.types(keyDefs.values(), DUMMY_SETTINGS);
+            if (tbl.cacheInfo().affinityNode())

Review comment:
       Cache filters should be checked as well as client nodes?

##########
File path: modules/geospatial/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GriH2SpatialBaseIndex.java
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.ignite.internal.processors.query.h2.opt;
+
+import java.util.HashSet;
+import org.h2.engine.Session;
+import org.h2.index.IndexType;
+import org.h2.index.SpatialIndex;
+import org.h2.index.SpatialTreeIndex;
+import org.h2.result.SearchRow;
+import org.h2.result.SortOrder;
+import org.h2.table.Column;
+import org.h2.table.IndexColumn;
+import org.h2.table.TableFilter;
+
+/** Base class for Geo-Spatial indexes to register in the H2 engine. */
+public abstract class GriH2SpatialBaseIndex extends GridH2IndexBase implements SpatialIndex {
+    /**
+     * Constructor.
+     *
+     * @param tbl  Table.
+     * @param name Index name.
+     * @param cols Indexed columns.
+     * @param type Index type.
+     */
+    protected GriH2SpatialBaseIndex(GridH2Table tbl, String name, IndexColumn[] cols, IndexType type) {
+        super(tbl, name, cols, type);
+    }
+
+    /** {@inheritDoc} */
+    @Override public H2CacheRow put(H2CacheRow row) {
+        throw new IllegalStateException("Must not be invoked.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean putx(H2CacheRow row) {
+        throw new IllegalStateException("Must not be invoked.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean removex(SearchRow row) {
+        throw new IllegalStateException("Must not be invoked.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public double getCost(Session ses, int[] masks, TableFilter[] filters, int filter,
+        SortOrder sortOrder, HashSet<Column> cols) {
+        return SpatialTreeIndex.getCostRangeIndex(masks, columns) / 10;

Review comment:
       Should be `/10d`?

##########
File path: modules/geospatial/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GriH2SpatialBaseIndex.java
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.ignite.internal.processors.query.h2.opt;
+
+import java.util.HashSet;
+import org.h2.engine.Session;
+import org.h2.index.IndexType;
+import org.h2.index.SpatialIndex;
+import org.h2.index.SpatialTreeIndex;
+import org.h2.result.SearchRow;
+import org.h2.result.SortOrder;
+import org.h2.table.Column;
+import org.h2.table.IndexColumn;
+import org.h2.table.TableFilter;
+
+/** Base class for Geo-Spatial indexes to register in the H2 engine. */
+public abstract class GriH2SpatialBaseIndex extends GridH2IndexBase implements SpatialIndex {

Review comment:
       Grid....

##########
File path: modules/geospatial/src/test/java/org/apache/ignite/internal/processors/query/h2/H2IndexingAbstractGeoSelfTest.java
##########
@@ -629,11 +663,11 @@ public Enemy(int campId, String name) {
     protected static class EnemyCamp implements Serializable {
         /** */
         @QuerySqlField(index = true)
-        Geometry coords;
+        final Geometry coords;

Review comment:
       protected?

##########
File path: modules/geospatial/src/test/java/org/apache/ignite/internal/processors/query/h2/H2IndexingAbstractGeoSelfTest.java
##########
@@ -76,6 +80,28 @@
     /** Segmented index flag. */
     private final boolean segmented;
 
+    /** Node that creates a Geo-Spatial index. */
+    @Parameterized.Parameter
+    public String crtNode;
+
+    /** Node that stars a query over a Geo-Spatial index. */
+    @Parameterized.Parameter(1)
+    public String qryNode;

Review comment:
       Please simplify to boolean couple and use them to select client node or server.

##########
File path: modules/geospatial/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GeoSpatialClientIndexDefinition.java
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.ignite.internal.processors.query.h2.opt;
+
+import java.util.LinkedHashMap;
+import java.util.List;
+import org.apache.ignite.internal.cache.query.index.IndexDefinition;
+import org.apache.ignite.internal.cache.query.index.IndexName;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyDefinition;
+import org.h2.table.IndexColumn;
+
+/**
+ * Definition of Geo-Spatial client index.
+ */
+public class GeoSpatialClientIndexDefinition implements IndexDefinition {
+    /** */
+    private final LinkedHashMap<String, IndexKeyDefinition> keyDefs;
+
+    /** */
+    private final IndexName idxName;
+
+    /** */
+    private final GridH2Table tbl;
+
+    /** */
+    private final List<IndexColumn> cols;

Review comment:
       We must be pretty sure to keep only required data. 
   Please recheck all these fields are used at client node.

##########
File path: modules/geospatial/src/test/java/org/apache/ignite/internal/processors/query/h2/H2IndexingAbstractGeoSelfTest.java
##########
@@ -122,11 +159,12 @@ protected H2IndexingAbstractGeoSelfTest(boolean segmented) {
      * @return Cache.
      * @throws Exception If failed.
      */
-    @SuppressWarnings("unchecked")
     protected <K, V> IgniteCache<K, V> createCache(String name, boolean partitioned, Class<?> keyCls, Class<?> valCls,
         boolean dynamicIdx) throws Exception {
         CacheConfiguration<K, V> ccfg = cacheConfig(name, partitioned, keyCls, valCls);
 
+        IgniteEx crtIgn = node(crtNode);

Review comment:
       use ternar operation here to chose between `cln` and `srv`

##########
File path: modules/geospatial/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GeoSpatialClientIndex.java
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.ignite.internal.processors.query.h2.opt;
+
+import java.util.List;
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.cache.query.index.AbstractIndex;
+import org.apache.ignite.internal.cache.query.index.Index;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexRow;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.util.lang.GridCursor;
+import org.h2.table.IndexColumn;
+import org.h2.table.TableFilter;
+import org.jetbrains.annotations.Nullable;
+import org.locationtech.jts.geom.Geometry;
+
+import static org.apache.ignite.internal.processors.query.h2.index.client.ClientInlineIndex.unsupported;
+
+/**
+ * Mock for client nodes to support Geo-Spatial indexes.
+ */
+public class GeoSpatialClientIndex extends AbstractIndex implements GeoSpatialIndex {
+    /** Index unique ID. */
+    private final UUID id = UUID.randomUUID();
+
+    /** */
+    private final String name;
+
+    /** */
+    private final GridH2Table tbl;
+
+    /** */
+    private final List<IndexColumn> cols;
+
+    /**
+     * @param def Index definition.
+     */
+    public GeoSpatialClientIndex(GeoSpatialClientIndexDefinition def) {
+        name = def.idxName().idxName();
+        tbl = def.tbl();
+        cols = def.cols();
+    }
+
+    /** {@inheritDoc} */
+    @Override public UUID id() {
+        return id;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String name() {
+        return name;
+    }
+
+    /** */
+    public GridH2Table tbl() {
+        return tbl;
+    }
+
+    /** */
+    public List<IndexColumn> cols() {
+        return cols;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean canHandle(CacheDataRow row) throws IgniteCheckedException {
+        throw unsupported();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onUpdate(
+        @Nullable CacheDataRow oldRow,
+        @Nullable CacheDataRow newRow,
+        boolean prevRowAvailable
+    ) {
+        throw unsupported();
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T extends Index> T unwrap(Class<T> clazz) {
+        if (clazz == null)
+            return null;
+
+        if (clazz.isAssignableFrom(getClass()))
+            return clazz.cast(this);
+
+        throw new IllegalArgumentException(
+            String.format("Cannot unwrap [%s] to [%s]", getClass().getName(), clazz.getName())
+        );
+    }

Review comment:
       Looks like a dupliccaton across all Index implementations.
   Could this be implemented as a default method?

##########
File path: modules/geospatial/src/test/java/org/apache/ignite/internal/processors/query/h2/H2IndexingAbstractGeoSelfTest.java
##########
@@ -86,15 +112,27 @@ protected H2IndexingAbstractGeoSelfTest(boolean segmented) {
     }
 
     /** {@inheritDoc} */
-    @Override protected int gridCount() {
-        return 3;
+    @Override public void beforeTest() throws Exception {
+        startGrids(3);

Review comment:
       assign `srv` node here as well as `cln` ant use it further

##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
##########
@@ -182,7 +182,7 @@
 
     /** Index manager. */
     @GridToStringExclude
-    private IndexProcessor idxMgr;
+    private final IndexProcessor idxProc;

Review comment:
       Good chance to make all fields final where possible




-- 
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@ignite.apache.org

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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9846: IGNITE-16604: Fix GeoSpatial indexes for client nodes

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9846:
URL: https://github.com/apache/ignite/pull/9846#discussion_r830886476



##########
File path: modules/geospatial/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GeoSpatialClientIndexDefinition.java
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.ignite.internal.processors.query.h2.opt;
+
+import java.util.LinkedHashMap;
+import java.util.List;
+import org.apache.ignite.internal.cache.query.index.IndexDefinition;
+import org.apache.ignite.internal.cache.query.index.IndexName;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyDefinition;
+import org.h2.table.IndexColumn;
+
+/**
+ * Definition of Geo-Spatial client index.
+ */
+public class GeoSpatialClientIndexDefinition implements IndexDefinition {
+    /** */
+    private final LinkedHashMap<String, IndexKeyDefinition> keyDefs;
+
+    /** */
+    private final IndexName idxName;
+
+    /** */
+    private final GridH2Table tbl;
+
+    /** */
+    private final List<IndexColumn> cols;

Review comment:
       Don't understand your concern, it looks like all those fields are required.




-- 
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@ignite.apache.org

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



[GitHub] [ignite] timoninmaxim merged pull request #9846: IGNITE-16604: Fix GeoSpatial indexes for client nodes

Posted by GitBox <gi...@apache.org>.
timoninmaxim merged pull request #9846:
URL: https://github.com/apache/ignite/pull/9846


   


-- 
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@ignite.apache.org

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