You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vo...@apache.org on 2017/03/09 12:32:38 UTC

[1/2] ignite git commit: Removed unnecessary GridQueryIndexType.

Repository: ignite
Updated Branches:
  refs/heads/ignite-4565-ddl f4a56fe67 -> d8d2ad8f9


Removed unnecessary GridQueryIndexType.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/157db2b6
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/157db2b6
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/157db2b6

Branch: refs/heads/ignite-4565-ddl
Commit: 157db2b6bfd40a76af3ee695665fe9c6c4a4b54e
Parents: f4a56fe
Author: devozerov <vo...@gridgain.com>
Authored: Thu Mar 9 15:04:09 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Mar 9 15:04:09 2017 +0300

----------------------------------------------------------------------
 .../configuration/CacheConfiguration.java       | 28 +++++------
 .../cache/query/GridCacheQueryManager.java      |  4 +-
 .../query/GridQueryIndexDescriptor.java         |  4 +-
 .../processors/query/GridQueryIndexType.java    | 32 ------------
 .../processors/query/GridQueryProcessor.java    | 51 ++++++++++----------
 .../resources/META-INF/classnames.properties    |  1 -
 .../processors/query/h2/IgniteH2Indexing.java   |  8 ++-
 .../h2/GridIndexingSpiAbstractSelfTest.java     |  6 +--
 8 files changed, 49 insertions(+), 85 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/157db2b6/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
index f0179ca..8ca48f3 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
@@ -66,7 +66,6 @@ import org.apache.ignite.cache.store.CacheStore;
 import org.apache.ignite.cache.store.CacheStoreSessionListener;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.processors.query.GridQueryIndexDescriptor;
-import org.apache.ignite.internal.processors.query.GridQueryIndexType;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.F;
@@ -78,9 +77,6 @@ import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.plugin.CachePluginConfiguration;
 import org.jetbrains.annotations.Nullable;
 
-import static org.apache.ignite.internal.processors.query.GridQueryIndexType.FULLTEXT;
-import static org.apache.ignite.internal.processors.query.GridQueryIndexType.GEO_SPATIAL;
-import static org.apache.ignite.internal.processors.query.GridQueryIndexType.SORTED;
 import static org.apache.ignite.internal.processors.query.GridQueryProcessor._VAL;
 import static org.apache.ignite.internal.processors.query.GridQueryProcessor.isGeometryClass;
 import static org.apache.ignite.internal.processors.query.GridQueryProcessor.isSqlType;
@@ -2220,7 +2216,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
         for (Map.Entry<String, GridQueryIndexDescriptor> idxEntry : desc.indexes().entrySet()) {
             GridQueryIndexDescriptor idx = idxEntry.getValue();
 
-            if (idx.type() == FULLTEXT) {
+            if (idx.type() == QueryIndexType.FULLTEXT) {
                 assert txtIdx == null;
 
                 txtIdx = new QueryIndex();
@@ -2238,7 +2234,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
 
                 QueryIndex sortedIdx = new QueryIndex();
 
-                sortedIdx.setIndexType(idx.type() == SORTED ? QueryIndexType.SORTED : QueryIndexType.GEOSPATIAL);
+                sortedIdx.setIndexType(idx.type());
 
                 LinkedHashMap<String, Boolean> fields = new LinkedHashMap<>();
 
@@ -2318,7 +2314,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
             if (parent == null && !key && isSqlType(cls)) { // We have to index primitive _val.
                 String idxName = _VAL + "_idx";
 
-                type.addIndex(idxName, isGeometryClass(cls) ? GEO_SPATIAL : SORTED);
+                type.addIndex(idxName, isGeometryClass(cls) ? QueryIndexType.GEOSPATIAL : QueryIndexType.SORTED);
 
                 type.addFieldToIndex(idxName, _VAL, 0, false);
             }
@@ -2338,13 +2334,13 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
             QueryGroupIndex grpIdx = cls.getAnnotation(QueryGroupIndex.class);
 
             if (grpIdx != null)
-                type.addIndex(grpIdx.name(), SORTED);
+                type.addIndex(grpIdx.name(), QueryIndexType.SORTED);
 
             QueryGroupIndex.List grpIdxList = cls.getAnnotation(QueryGroupIndex.List.class);
 
             if (grpIdxList != null && !F.isEmpty(grpIdxList.value())) {
                 for (QueryGroupIndex idx : grpIdxList.value())
-                    type.addIndex(idx.name(), SORTED);
+                    type.addIndex(idx.name(), QueryIndexType.SORTED);
             }
         }
 
@@ -2417,7 +2413,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
             if (sqlAnn.index()) {
                 String idxName = prop.alias() + "_idx";
 
-                desc.addIndex(idxName, isGeometryClass(prop.type()) ? GEO_SPATIAL : SORTED);
+                desc.addIndex(idxName, isGeometryClass(prop.type()) ? QueryIndexType.GEOSPATIAL: QueryIndexType.SORTED);
 
                 desc.addFieldToIndex(idxName, prop.fullName(), 0, sqlAnn.descending());
             }
@@ -2506,7 +2502,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
          * @param type Index type.
          * @return Index descriptor.
          */
-        public IndexDescriptor addIndex(String idxName, GridQueryIndexType type) {
+        public IndexDescriptor addIndex(String idxName, QueryIndexType type) {
             IndexDescriptor idx = new IndexDescriptor(type);
 
             if (indexes.put(idxName, idx) != null)
@@ -2528,7 +2524,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
             IndexDescriptor desc = indexes.get(idxName);
 
             if (desc == null)
-                desc = addIndex(idxName, SORTED);
+                desc = addIndex(idxName, QueryIndexType.SORTED);
 
             desc.addField(field, orderNum, descending);
         }
@@ -2540,7 +2536,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
          */
         public void addFieldToTextIndex(String field) {
             if (fullTextIdx == null) {
-                fullTextIdx = new IndexDescriptor(FULLTEXT);
+                fullTextIdx = new IndexDescriptor(QueryIndexType.FULLTEXT);
 
                 indexes.put(null, fullTextIdx);
             }
@@ -2640,12 +2636,12 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
         private Collection<String> descendings;
 
         /** */
-        private final GridQueryIndexType type;
+        private final QueryIndexType type;
 
         /**
          * @param type Type.
          */
-        private IndexDescriptor(GridQueryIndexType type) {
+        private IndexDescriptor(QueryIndexType type) {
             assert type != null;
 
             this.type = type;
@@ -2685,7 +2681,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
         }
 
         /** {@inheritDoc} */
-        @Override public GridQueryIndexType type() {
+        @Override public QueryIndexType type() {
             return type;
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/157db2b6/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
index 66ac16d..65c7508 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
@@ -47,6 +47,7 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.cache.QueryIndex;
+import org.apache.ignite.cache.QueryIndexType;
 import org.apache.ignite.cache.query.QueryMetrics;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.CacheConfiguration;
@@ -82,7 +83,6 @@ import org.apache.ignite.internal.processors.datastructures.SetItemKey;
 import org.apache.ignite.internal.processors.platform.cache.PlatformCacheEntryFilter;
 import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata;
 import org.apache.ignite.internal.processors.query.GridQueryIndexDescriptor;
-import org.apache.ignite.internal.processors.query.GridQueryIndexType;
 import org.apache.ignite.internal.processors.query.GridQueryProcessor;
 import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
 import org.apache.ignite.internal.processors.task.GridInternal;
@@ -2497,7 +2497,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
                             GridQueryIndexDescriptor desc = e.getValue();
 
                             // Add only SQL SORTED indexes.
-                            if (desc.type() == GridQueryIndexType.SORTED) {
+                            if (desc.type() == QueryIndexType.SORTED) {
                                 Collection<String> idxFields = new LinkedList<>();
                                 Collection<String> descendings = new LinkedList<>();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/157db2b6/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexDescriptor.java
index fe58112..134a61c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexDescriptor.java
@@ -17,6 +17,8 @@
 
 package org.apache.ignite.internal.processors.query;
 
+import org.apache.ignite.cache.QueryIndexType;
+
 import java.util.Collection;
 
 /**
@@ -46,5 +48,5 @@ public interface GridQueryIndexDescriptor {
      *
      * @return Type.
      */
-    public GridQueryIndexType type();
+    public QueryIndexType type();
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/157db2b6/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexType.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexType.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexType.java
deleted file mode 100644
index f59301c..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexType.java
+++ /dev/null
@@ -1,32 +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.ignite.internal.processors.query;
-
-/**
- * Index types.
- */
-public enum GridQueryIndexType {
-    /** Sorted SQL index. */
-    SORTED,
-
-    /** Spatial SQL index. */
-    GEO_SPATIAL,
-
-    /** Fulltext index. */
-    FULLTEXT
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/157db2b6/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index 01c7122..cb46612 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -101,9 +101,6 @@ import org.jsr166.ConcurrentHashMap8;
 
 import static org.apache.ignite.events.EventType.EVT_CACHE_QUERY_EXECUTED;
 import static org.apache.ignite.internal.IgniteComponentType.INDEXING;
-import static org.apache.ignite.internal.processors.query.GridQueryIndexType.FULLTEXT;
-import static org.apache.ignite.internal.processors.query.GridQueryIndexType.GEO_SPATIAL;
-import static org.apache.ignite.internal.processors.query.GridQueryIndexType.SORTED;
 
 /**
  * Indexing processor.
@@ -1431,13 +1428,14 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             if (idxName == null)
                 idxName = propName + "_idx";
 
-            if (idxOrder == 0) // Add index only on the first field.
-                d.addIndex(idxName, isGeometryClass(propCls) ? GEO_SPATIAL : SORTED);
-
             if (idxType == IndexType.TEXT)
                 d.addFieldToTextIndex(propName);
-            else
+            else {
+                if (idxOrder == 0) // Add index only on the first field.
+                    d.addIndex(idxName, isGeometryClass(propCls) ? QueryIndexType.GEOSPATIAL : QueryIndexType.SORTED);
+
                 d.addFieldToIndex(idxName, propName, idxOrder, idxType == IndexType.DESC);
+            }
         }
     }
 
@@ -1462,7 +1460,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
             String idxName = prop.name() + "_idx";
 
-            d.addIndex(idxName, isGeometryClass(prop.type()) ? GEO_SPATIAL : SORTED);
+            d.addIndex(idxName, isGeometryClass(prop.type()) ? QueryIndexType.GEOSPATIAL : QueryIndexType.SORTED);
 
             d.addFieldToIndex(idxName, prop.name(), 0, false);
         }
@@ -1474,7 +1472,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
             String idxName = prop.name() + "_idx";
 
-            d.addIndex(idxName, isGeometryClass(prop.type()) ? GEO_SPATIAL : SORTED);
+            d.addIndex(idxName, isGeometryClass(prop.type()) ? QueryIndexType.GEOSPATIAL : QueryIndexType.SORTED);
 
             d.addFieldToIndex(idxName, prop.name(), 0, true);
         }
@@ -1495,19 +1493,23 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
                 LinkedHashMap<String, IgniteBiTuple<Class<?>, Boolean>> idxFields = entry.getValue();
 
-                int order = 0;
+                if (!idxFields.isEmpty()) {
+                    d.addIndex(idxName, QueryIndexType.SORTED);
 
-                for (Map.Entry<String, IgniteBiTuple<Class<?>, Boolean>> idxField : idxFields.entrySet()) {
-                    BinaryProperty prop = buildBinaryProperty(idxField.getKey(), idxField.getValue().get1(), aliases,
-                        null);
+                    int order = 0;
 
-                    d.addProperty(prop, false);
+                    for (Map.Entry<String, IgniteBiTuple<Class<?>, Boolean>> idxField : idxFields.entrySet()) {
+                        BinaryProperty prop = buildBinaryProperty(idxField.getKey(), idxField.getValue().get1(), aliases,
+                            null);
 
-                    Boolean descending = idxField.getValue().get2();
+                        d.addProperty(prop, false);
 
-                    d.addFieldToIndex(idxName, prop.name(), order, descending != null && descending);
+                        Boolean descending = idxField.getValue().get2();
 
-                    order++;
+                        d.addFieldToIndex(idxName, prop.name(), order, descending != null && descending);
+
+                        order++;
+                    }
                 }
             }
         }
@@ -1624,7 +1626,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                 QueryIndexType idxTyp = idx.getIndexType();
 
                 if (idxTyp == QueryIndexType.SORTED || idxTyp == QueryIndexType.GEOSPATIAL) {
-                    d.addIndex(idxName, idxTyp == QueryIndexType.SORTED ? SORTED : GEO_SPATIAL);
+                    d.addIndex(idxName, idxTyp);
 
                     int i = 0;
 
@@ -2468,7 +2470,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
          * @return Index descriptor.
          * @throws IgniteCheckedException In case of error.
          */
-        public IndexDescriptor addIndex(String idxName, GridQueryIndexType type) throws IgniteCheckedException {
+        public IndexDescriptor addIndex(String idxName, QueryIndexType type) throws IgniteCheckedException {
             IndexDescriptor idx = new IndexDescriptor(type);
 
             if (indexes.put(idxName, idx) != null)
@@ -2490,8 +2492,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             boolean descending) throws IgniteCheckedException {
             IndexDescriptor desc = indexes.get(idxName);
 
-            if (desc == null)
-                desc = addIndex(idxName, SORTED);
+            assert desc != null;
 
             desc.addField(field, orderNum, descending);
         }
@@ -2503,7 +2504,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
          */
         public void addFieldToTextIndex(String field) {
             if (fullTextIdx == null)
-                fullTextIdx = new IndexDescriptor(FULLTEXT);
+                fullTextIdx = new IndexDescriptor(QueryIndexType.FULLTEXT);
 
             fullTextIdx.addField(field, 0, false);
         }
@@ -2658,12 +2659,12 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         private Collection<String> descendings;
 
         /** */
-        private final GridQueryIndexType type;
+        private final QueryIndexType type;
 
         /**
          * @param type Type.
          */
-        private IndexDescriptor(GridQueryIndexType type) {
+        private IndexDescriptor(QueryIndexType type) {
             assert type != null;
 
             this.type = type;
@@ -2703,7 +2704,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         }
 
         /** {@inheritDoc} */
-        @Override public GridQueryIndexType type() {
+        @Override public QueryIndexType type() {
             return type;
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/157db2b6/modules/core/src/main/resources/META-INF/classnames.properties
----------------------------------------------------------------------
diff --git a/modules/core/src/main/resources/META-INF/classnames.properties b/modules/core/src/main/resources/META-INF/classnames.properties
index 02bad40..0ad74a9 100644
--- a/modules/core/src/main/resources/META-INF/classnames.properties
+++ b/modules/core/src/main/resources/META-INF/classnames.properties
@@ -1276,7 +1276,6 @@ org.apache.ignite.internal.processors.platform.utils.PlatformFutureUtils$Interna
 org.apache.ignite.internal.processors.platform.websession.PlatformDotNetSessionLockProcessor
 org.apache.ignite.internal.processors.platform.websession.PlatformDotNetSessionSetAndUnlockProcessor
 org.apache.ignite.internal.processors.query.GridQueryFieldMetadata
-org.apache.ignite.internal.processors.query.GridQueryIndexType
 org.apache.ignite.internal.processors.query.GridQueryProcessor$3
 org.apache.ignite.internal.processors.query.GridQueryProcessor$4
 org.apache.ignite.internal.processors.query.GridQueryProcessor$5

http://git-wip-us.apache.org/repos/asf/ignite/blob/157db2b6/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
index c7077a8..7635074 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
@@ -59,6 +59,7 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.cache.CacheMemoryMode;
+import org.apache.ignite.cache.QueryIndexType;
 import org.apache.ignite.cache.query.QueryCancelledException;
 import org.apache.ignite.cache.query.QueryCursor;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
@@ -185,9 +186,6 @@ import static org.apache.ignite.IgniteSystemProperties.getString;
 import static org.apache.ignite.internal.processors.cache.query.GridCacheQueryType.SQL;
 import static org.apache.ignite.internal.processors.cache.query.GridCacheQueryType.SQL_FIELDS;
 import static org.apache.ignite.internal.processors.cache.query.GridCacheQueryType.TEXT;
-import static org.apache.ignite.internal.processors.query.GridQueryIndexType.FULLTEXT;
-import static org.apache.ignite.internal.processors.query.GridQueryIndexType.GEO_SPATIAL;
-import static org.apache.ignite.internal.processors.query.GridQueryIndexType.SORTED;
 import static org.apache.ignite.internal.processors.query.h2.opt.GridH2AbstractKeyValueRow.KEY_COL;
 import static org.apache.ignite.internal.processors.query.h2.opt.GridH2QueryType.LOCAL;
 import static org.apache.ignite.internal.processors.query.h2.opt.GridH2QueryType.PREPARE;
@@ -2657,7 +2655,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                         idx.descending(field) ? SortOrder.DESCENDING : SortOrder.ASCENDING));
                 }
 
-                if (idx.type() == SORTED) {
+                if (idx.type() == QueryIndexType.SORTED) {
                     // We don't care about number of fields in affinity index, just affinity key must be the first.
                     affIdxFound |= affCol != null && equal(cols.get(0), affCol);
 
@@ -2665,7 +2663,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
                     idxs.add(new GridH2TreeIndex(name, tbl, false, cols));
                 }
-                else if (idx.type() == GEO_SPATIAL)
+                else if (idx.type() == QueryIndexType.GEOSPATIAL)
                     idxs.add(createH2SpatialIndex(tbl, name, cols.toArray(new IndexColumn[cols.size()])));
                 else
                     throw new IllegalStateException("Index type: " + idx.type());

http://git-wip-us.apache.org/repos/asf/ignite/blob/157db2b6/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
index e670666..33769f3 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
@@ -27,12 +27,12 @@ import java.util.List;
 import java.util.Map;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cache.QueryIndexType;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.CacheObjectContext;
 import org.apache.ignite.internal.processors.query.GridQueryFieldsResult;
 import org.apache.ignite.internal.processors.query.GridQueryIndexDescriptor;
-import org.apache.ignite.internal.processors.query.GridQueryIndexType;
 import org.apache.ignite.internal.processors.query.GridQueryProperty;
 import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
 import org.apache.ignite.internal.util.typedef.F;
@@ -502,8 +502,8 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
         }
 
         /** {@inheritDoc} */
-        @Override public GridQueryIndexType type() {
-            return GridQueryIndexType.FULLTEXT;
+        @Override public QueryIndexType type() {
+            return QueryIndexType.FULLTEXT;
         }
     }
 


[2/2] ignite git commit: WIP on general wire-up.

Posted by vo...@apache.org.
WIP on general wire-up.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/d8d2ad8f
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/d8d2ad8f
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/d8d2ad8f

Branch: refs/heads/ignite-4565-ddl
Commit: d8d2ad8f93efedd4787ec04a9886b28266f7ac8d
Parents: 157db2b
Author: devozerov <vo...@gridgain.com>
Authored: Thu Mar 9 15:32:28 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Mar 9 15:32:28 2017 +0300

----------------------------------------------------------------------
 .../processors/query/GridQueryProcessor.java    | 42 +++++++++++++++++++-
 1 file changed, 40 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d8d2ad8f/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index cb46612..bb237ab 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -37,9 +37,12 @@ import java.util.Map;
 import java.util.Set;
 import java.util.TreeSet;
 import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
 import javax.cache.Cache;
 import javax.cache.CacheException;
 import org.apache.ignite.IgniteCheckedException;
@@ -99,6 +102,7 @@ import org.apache.ignite.spi.indexing.IndexingQueryFilter;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
 
+import static java.lang.Enum.valueOf;
 import static org.apache.ignite.events.EventType.EVT_CACHE_QUERY_EXECUTED;
 import static org.apache.ignite.internal.IgniteComponentType.INDEXING;
 
@@ -341,6 +345,8 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                         altTypeId = new TypeId(ccfg.getName(), ctx.cacheObjects().typeId(qryEntity.getValueType()));
                     }
 
+                    desc.onInitialStateReady();
+
                     addTypeByName(ccfg, desc);
                     types.put(typeId, desc);
 
@@ -434,6 +440,8 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                         altTypeId = new TypeId(ccfg.getName(), ctx.cacheObjects().typeId(meta.getValueType()));
                     }
 
+                    desc.onInitialStateReady();
+
                     addTypeByName(ccfg, desc);
                     types.put(typeId, desc);
 
@@ -2327,6 +2335,9 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         @GridToStringInclude
         private final Map<String, IndexDescriptor> indexes = new HashMap<>();
 
+        /** Index state manager. */
+        private final IndexStateManager idxState = new IndexStateManager();
+
         /** */
         private IndexDescriptor fullTextIdx;
 
@@ -2488,8 +2499,8 @@ public class GridQueryProcessor extends GridProcessorAdapter {
          * @param descending Sorting order.
          * @throws IgniteCheckedException If failed.
          */
-        public void addFieldToIndex(String idxName, String field, int orderNum,
-            boolean descending) throws IgniteCheckedException {
+        public void addFieldToIndex(String idxName, String field, int orderNum, boolean descending)
+            throws IgniteCheckedException {
             IndexDescriptor desc = indexes.get(idxName);
 
             assert desc != null;
@@ -2623,6 +2634,13 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         }
 
         /**
+         * Callback invoked when initial type state is ready.
+         */
+        public void onInitialStateReady() {
+            idxState.onInitialStateReady(indexes);
+        }
+
+        /**
          * Initiate asynchronous index creation.
          *
          * @param idx Index description.
@@ -2715,6 +2733,26 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * Index state manager.
+     */
+    private static class IndexStateManager {
+        /** Indexes. */
+        private final Map<String, IndexDescriptor> idxs = new ConcurrentHashMap<>();
+
+        /** RW lock. */
+        private final ReadWriteLock lock = new ReentrantReadWriteLock();
+
+        /**
+         * Callback invoked when original index state is ready.
+         *
+         * @param idxs Indexes.
+         */
+        public void onInitialStateReady(Map<String, IndexDescriptor> idxs) {
+            this.idxs.putAll(idxs);
+        }
+    }
+
+    /**
      * Identifying TypeDescriptor by space and value class.
      */
     private static class TypeId {