You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2017/05/24 13:36:23 UTC

[18/31] ignite git commit: IGNITE-5281: Indexing: changed "space" to "cacheName". No more "spaces". This closes #1992.

IGNITE-5281: Indexing: changed "space" to "cacheName". No more "spaces". This closes #1992.

:quit
:quit
mdules/web-console/frontend/app/modules/states/configuration/summary/summary-zipper.service.js~HEAD


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

Branch: refs/heads/ignite-5075
Commit: cbf0b2a5873395b127e05b411136f2d30f449573
Parents: 018b25b
Author: devozerov <vo...@gridgain.com>
Authored: Tue May 23 20:37:55 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Tue May 23 20:38:49 2017 +0300

----------------------------------------------------------------------
 .../ignite/internal/jdbc2/JdbcConnection.java   |   3 +-
 .../managers/indexing/GridIndexingManager.java  |  19 +-
 .../cache/query/GridCacheQueryManager.java      |  32 +--
 .../processors/query/GridQueryIndexing.java     |  85 +++---
 .../processors/query/GridQueryProcessor.java    | 153 +++++------
 .../processors/query/QueryIndexKey.java         |  20 +-
 .../query/QueryTypeDescriptorImpl.java          |  16 +-
 .../processors/query/QueryTypeIdKey.java        |  26 +-
 .../processors/query/QueryTypeNameKey.java      |  12 +-
 .../internal/processors/query/QueryUtils.java   |  14 +-
 .../schema/SchemaIndexCacheVisitorImpl.java     |  12 +-
 .../operation/SchemaAbstractOperation.java      |   2 +-
 .../spi/indexing/IndexingQueryFilter.java       |   8 +-
 .../apache/ignite/spi/indexing/IndexingSpi.java |  17 +-
 .../spi/indexing/noop/NoopIndexingSpi.java      |   6 +-
 .../IgniteTxExceptionAbstractSelfTest.java      |   6 +-
 .../cache/query/IndexingSpiQuerySelfTest.java   |  14 +-
 .../cache/query/IndexingSpiQueryTxSelfTest.java |   6 +-
 .../query/h2/DmlStatementsProcessor.java        |  51 ++--
 .../processors/query/h2/IgniteH2Indexing.java   | 258 +++++++++----------
 .../query/h2/database/H2PkHashIndex.java        |   4 +-
 .../query/h2/database/H2TreeIndex.java          |   4 +-
 .../query/h2/opt/GridH2IndexBase.java           |  10 +-
 .../processors/query/h2/opt/GridH2Table.java    |   2 +-
 .../query/h2/opt/GridH2TreeIndex.java           |   2 +-
 .../query/h2/opt/GridLuceneIndex.java           |  15 +-
 .../query/h2/twostep/GridMapQueryExecutor.java  |   2 +-
 .../h2/twostep/GridReduceQueryExecutor.java     |  30 +--
 .../DynamicIndexAbstractBasicSelfTest.java      |  18 +-
 .../DynamicIndexAbstractConcurrentSelfTest.java |   8 +-
 .../query/IgniteQueryDedicatedPoolTest.java     |   6 +-
 .../h2/GridIndexingSpiAbstractSelfTest.java     | 136 +++++-----
 .../query/h2/sql/GridQueryParsingTest.java      |   2 +-
 33 files changed, 491 insertions(+), 508 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/cbf0b2a5/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcConnection.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcConnection.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcConnection.java
index 8da385a..f6f79fb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcConnection.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcConnection.java
@@ -47,7 +47,6 @@ import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteClientDisconnectedException;
-import org.apache.ignite.IgniteCompute;
 import org.apache.ignite.IgniteDataStreamer;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteJdbcDriver;
@@ -708,7 +707,7 @@ public class JdbcConnection implements Connection {
     @Override public void setSchema(String schema) throws SQLException {
         assert ignite instanceof IgniteEx;
 
-        cacheName = ((IgniteEx)ignite).context().query().space(schema);
+        cacheName = ((IgniteEx)ignite).context().query().cacheName(schema);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbf0b2a5/modules/core/src/main/java/org/apache/ignite/internal/managers/indexing/GridIndexingManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/indexing/GridIndexingManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/indexing/GridIndexingManager.java
index a60cdbd..6d29604 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/indexing/GridIndexingManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/indexing/GridIndexingManager.java
@@ -79,14 +79,15 @@ public class GridIndexingManager extends GridManagerAdapter<IndexingSpi> {
     /**
      * Writes key-value pair to index.
      *
-     * @param space Space.
+     * @param cacheName Cache name.
      * @param key Key.
      * @param val Value.
      * @param expirationTime Expiration time or 0 if never expires.
      * @throws IgniteCheckedException In case of error.
      */
     @SuppressWarnings("unchecked")
-    public <K, V> void store(final String space, final K key, final V val, long expirationTime) throws IgniteCheckedException {
+    public <K, V> void store(final String cacheName, final K key, final V val, long expirationTime)
+        throws IgniteCheckedException {
         assert key != null;
         assert val != null;
         assert enabled();
@@ -98,7 +99,7 @@ public class GridIndexingManager extends GridManagerAdapter<IndexingSpi> {
             if (log.isDebugEnabled())
                 log.debug("Storing key to cache query index [key=" + key + ", value=" + val + "]");
 
-            getSpi().store(space, key, val, expirationTime);
+            getSpi().store(cacheName, key, val, expirationTime);
         }
         finally {
             busyLock.leaveBusy();
@@ -106,12 +107,12 @@ public class GridIndexingManager extends GridManagerAdapter<IndexingSpi> {
     }
 
     /**
-     * @param space Space.
+     * @param cacheName Cache name.
      * @param key Key.
      * @throws IgniteCheckedException Thrown in case of any errors.
      */
     @SuppressWarnings("unchecked")
-    public void remove(String space, Object key) throws IgniteCheckedException {
+    public void remove(String cacheName, Object key) throws IgniteCheckedException {
         assert key != null;
         assert enabled();
 
@@ -119,7 +120,7 @@ public class GridIndexingManager extends GridManagerAdapter<IndexingSpi> {
             throw new IllegalStateException("Failed to remove from index (grid is stopping).");
 
         try {
-            getSpi().remove(space, key);
+            getSpi().remove(cacheName, key);
         }
         finally {
             busyLock.leaveBusy();
@@ -127,14 +128,14 @@ public class GridIndexingManager extends GridManagerAdapter<IndexingSpi> {
     }
 
     /**
-     * @param space Space.
+     * @param cacheName Cache name.
      * @param params Parameters collection.
      * @param filters Filters.
      * @return Query result.
      * @throws IgniteCheckedException If failed.
      */
     @SuppressWarnings("unchecked")
-    public IgniteSpiCloseableIterator<?> query(String space, Collection<Object> params, IndexingQueryFilter filters)
+    public IgniteSpiCloseableIterator<?> query(String cacheName, Collection<Object> params, IndexingQueryFilter filters)
         throws IgniteCheckedException {
         if (!enabled())
             throw new IgniteCheckedException("Indexing SPI is not configured.");
@@ -143,7 +144,7 @@ public class GridIndexingManager extends GridManagerAdapter<IndexingSpi> {
             throw new IllegalStateException("Failed to execute query (grid is stopping).");
 
         try {
-            final Iterator<?> res = getSpi().query(space, params, filters);
+            final Iterator<?> res = getSpi().query(cacheName, params, filters);
 
             if (res == null)
                 return new GridEmptyCloseableIterator<>();

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbf0b2a5/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 0a789ae..07545a5 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
@@ -173,7 +173,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
     private GridQueryProcessor qryProc;
 
     /** */
-    private String space;
+    private String cacheName;
 
     /** */
     private int maxIterCnt;
@@ -221,9 +221,9 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
 
         qryProc = cctx.kernalContext().query();
 
-        space = cctx.name();
+        cacheName = cctx.name();
 
-        enabled = qryProcEnabled || (isIndexingSpiEnabled() && !CU.isSystemCache(space));
+        enabled = qryProcEnabled || (isIndexingSpiEnabled() && !CU.isSystemCache(cacheName));
 
         maxIterCnt = ccfg.getMaxQueryIteratorsCount();
 
@@ -409,11 +409,11 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
 
                 Object val0 = unwrapIfNeeded(val, coctx);
 
-                cctx.kernalContext().indexing().store(space, key0, val0, expirationTime);
+                cctx.kernalContext().indexing().store(cacheName, key0, val0, expirationTime);
             }
 
             if(qryProcEnabled)
-                qryProc.store(space, key, partId, prevVal, prevVer, val, ver, expirationTime, link);
+                qryProc.store(cacheName, key, partId, prevVal, prevVer, val, ver, expirationTime, link);
         }
         finally {
             invalidateResultCache();
@@ -443,12 +443,12 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
             if (isIndexingSpiEnabled()) {
                 Object key0 = unwrapIfNeeded(key, cctx.cacheObjectContext());
 
-                cctx.kernalContext().indexing().remove(space, key0);
+                cctx.kernalContext().indexing().remove(cacheName, key0);
             }
 
             // val may be null if we have no previous value. We should not call processor in this case.
             if(qryProcEnabled && val != null)
-                qryProc.remove(space, key, partId, val, ver);
+                qryProc.remove(cacheName, key, partId, val, ver);
         }
         finally {
             invalidateResultCache();
@@ -628,7 +628,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
                             taskName));
                     }
 
-                    iter = qryProc.queryText(space, qry.clause(), qry.queryClassName(), filter(qry));
+                    iter = qryProc.queryText(cacheName, qry.clause(), qry.queryClassName(), filter(qry));
 
                     break;
 
@@ -739,7 +739,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
 
         try {
             if (qry.type() == SPI) {
-                IgniteSpiCloseableIterator<?> iter = cctx.kernalContext().indexing().query(space, F.asList(args),
+                IgniteSpiCloseableIterator<?> iter = cctx.kernalContext().indexing().query(cacheName, F.asList(args),
                     filter(qry));
 
                 res.onDone(iter);
@@ -1907,7 +1907,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
             // Remote nodes that have current cache.
             Collection<ClusterNode> nodes = F.view(cctx.discovery().remoteNodes(), new P1<ClusterNode>() {
                 @Override public boolean apply(ClusterNode n) {
-                    return cctx.kernalContext().discovery().cacheAffinityNode(n, space);
+                    return cctx.kernalContext().discovery().cacheAffinityNode(n, cacheName);
                 }
             });
 
@@ -1945,7 +1945,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
             Collection<GridCacheSqlMetadata> col = new ArrayList<>(map.size());
 
             // Metadata for current cache must be first in list.
-            col.add(new CacheSqlMetadata(map.remove(space)));
+            col.add(new CacheSqlMetadata(map.remove(cacheName)));
 
             for (Collection<CacheSqlMetadata> metas : map.values())
                 col.add(new CacheSqlMetadata(metas));
@@ -1969,10 +1969,10 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
             return null;
 
         return new IndexingQueryFilter() {
-            @Nullable @Override public IgniteBiPredicate<K, V> forSpace(final String spaceName) {
+            @Nullable @Override public IgniteBiPredicate<K, V> forCache(final String cacheName) {
                 final GridKernalContext ctx = cctx.kernalContext();
 
-                final GridCacheAdapter<Object, Object> cache = ctx.cache().internalCache(spaceName);
+                final GridCacheAdapter<Object, Object> cache = ctx.cache().internalCache(cacheName);
 
                 if (cache.context().isReplicated() || cache.configuration().getBackups() == 0)
                     return null;
@@ -2016,10 +2016,10 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
     /**
      * FOR TESTING ONLY
      *
-     * @return Indexing space for this query manager.
+     * @return Cache name for this query manager.
      */
-    public String space() {
-        return space;
+    public String cacheName() {
+        return cacheName;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbf0b2a5/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
index bfa223e..1f5e701 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
@@ -89,14 +89,14 @@ public interface GridQueryIndexing {
     /**
      * Perform a MERGE statement using data streamer as receiver.
      *
-     * @param spaceName Space name.
+     * @param cacheName Cache name.
      * @param qry Query.
      * @param params Query parameters.
      * @param streamer Data streamer to feed data to.
      * @return Query result.
      * @throws IgniteCheckedException If failed.
      */
-    public long streamUpdateQuery(final String spaceName, final String qry, @Nullable final Object[] params,
+    public long streamUpdateQuery(String cacheName, String qry, @Nullable Object[] params,
         IgniteDataStreamer<?, ?> streamer) throws IgniteCheckedException;
 
     /**
@@ -104,7 +104,7 @@ public interface GridQueryIndexing {
      *
      * @param cctx Cache context.
      * @param qry Query.
-     * @param filter Space name and key filter.
+     * @param filter Cache name and key filter.
      * @param keepBinary Keep binary flag.
      * @return Cursor.
      */
@@ -116,7 +116,7 @@ public interface GridQueryIndexing {
      *
      * @param cctx Cache context.
      * @param qry Query.
-     * @param filter Space name and key filter.
+     * @param filter Cache name and key filter.
      * @param cancel Query cancel.
      * @return Cursor.
      */
@@ -126,84 +126,84 @@ public interface GridQueryIndexing {
     /**
      * Executes text query.
      *
-     * @param spaceName Space name.
+     * @param cacheName Cache name.
      * @param qry Text query.
      * @param typeName Type name.
-     * @param filter Space name and key filter.
+     * @param filter Cache name and key filter.
      * @return Queried rows.
      * @throws IgniteCheckedException If failed.
      */
-    public <K, V> GridCloseableIterator<IgniteBiTuple<K, V>> queryLocalText(String spaceName, String qry,
+    public <K, V> GridCloseableIterator<IgniteBiTuple<K, V>> queryLocalText(String cacheName, String qry,
         String typeName, IndexingQueryFilter filter) throws IgniteCheckedException;
 
     /**
      * Create new index locally.
      *
-     * @param spaceName Space name.
+     * @param cacheName Cache name.
      * @param tblName Table name.
      * @param idxDesc Index descriptor.
      * @param ifNotExists Ignore operation if index exists (instead of throwing an error).
      * @param cacheVisitor Cache visitor
      * @throws IgniteCheckedException if failed.
      */
-    public void dynamicIndexCreate(String spaceName, String tblName, QueryIndexDescriptorImpl idxDesc,
+    public void dynamicIndexCreate(String cacheName, String tblName, QueryIndexDescriptorImpl idxDesc,
         boolean ifNotExists, SchemaIndexCacheVisitor cacheVisitor) throws IgniteCheckedException;
 
     /**
-     * Remove index from the space.
+     * Remove index from the cache.
      *
-     * @param spaceName Space name.
+     * @param cacheName cache name.
      * @param idxName Index name.
      * @param ifExists Ignore operation if index does not exist (instead of throwing an error).
      * @throws IgniteCheckedException If failed.
      */
     @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
-    public void dynamicIndexDrop(String spaceName, String idxName, boolean ifExists)
+    public void dynamicIndexDrop(String cacheName, String idxName, boolean ifExists)
         throws IgniteCheckedException;
 
     /**
      * Registers cache.
      *
-     * @param spaceName Space name.
+     * @param cacheName Cache name.
      * @param cctx Cache context.
      * @param ccfg Cache configuration.
      * @throws IgniteCheckedException If failed.
      */
-    public void registerCache(String spaceName, GridCacheContext<?,?> cctx, CacheConfiguration<?,?> ccfg)
+    public void registerCache(String cacheName, GridCacheContext<?,?> cctx, CacheConfiguration<?,?> ccfg)
         throws IgniteCheckedException;
 
     /**
      * Unregisters cache.
      *
-     * @param spaceName Space name.
+     * @param cacheName Cache name.
      * @throws IgniteCheckedException If failed to drop cache schema.
      */
-    public void unregisterCache(String spaceName) throws IgniteCheckedException;
+    public void unregisterCache(String cacheName) throws IgniteCheckedException;
 
     /**
      * Registers type if it was not known before or updates it otherwise.
      *
-     * @param spaceName Space name.
+     * @param cacheName Cache name.
      * @param desc Type descriptor.
      * @throws IgniteCheckedException If failed.
      * @return {@code True} if type was registered, {@code false} if for some reason it was rejected.
      */
-    public boolean registerType(String spaceName, GridQueryTypeDescriptor desc) throws IgniteCheckedException;
+    public boolean registerType(String cacheName, GridQueryTypeDescriptor desc) throws IgniteCheckedException;
 
     /**
      * Unregisters type and removes all corresponding data.
      *
-     * @param spaceName Space name.
+     * @param cacheName Cache name.
      * @param typeName Type name.
      * @throws IgniteCheckedException If failed.
      */
-    public void unregisterType(String spaceName, String typeName) throws IgniteCheckedException;
+    public void unregisterType(String cacheName, String typeName) throws IgniteCheckedException;
 
     /**
-     * Updates index. Note that key is unique for space, so if space contains multiple indexes
+     * Updates index. Note that key is unique for cache, so if cache contains multiple indexes
      * the key should be removed from indexes other than one being updated.
      *
-     * @param spaceName Space name.
+     * @param cacheName Cache name.
      * @param typeName Type name.
      * @param key Key.
      * @param val Value.
@@ -211,47 +211,36 @@ public interface GridQueryIndexing {
      * @param expirationTime Expiration time or 0 if never expires.
      * @throws IgniteCheckedException If failed.
      */
-    public void store(String spaceName,
-        String typeName,
-        KeyCacheObject key,
-        int partId,
-        CacheObject val,
-        GridCacheVersion ver,
-        long expirationTime,
-        long link) throws IgniteCheckedException;
+    public void store(String cacheName, String typeName, KeyCacheObject key, int partId, CacheObject val,
+        GridCacheVersion ver, long expirationTime, long link) throws IgniteCheckedException;
 
     /**
      * Removes index entry by key.
      *
-     * @param spaceName Space name.
+     * @param cacheName Cache name.
      * @param key Key.
      * @param val Value.
      * @throws IgniteCheckedException If failed.
      */
-    public void remove(String spaceName,
-        GridQueryTypeDescriptor type,
-        KeyCacheObject key,
-        int partId,
-        CacheObject val,
+    public void remove(String cacheName, GridQueryTypeDescriptor type, KeyCacheObject key, int partId, CacheObject val,
         GridCacheVersion ver) throws IgniteCheckedException;
 
     /**
      * Rebuilds all indexes of given type from hash index.
      *
-     * @param spaceName Space name.
+     * @param cacheName Cache name.
      * @param type Type descriptor.
      * @throws IgniteCheckedException If failed.
      */
-    public void rebuildIndexesFromHash(String spaceName,
-        GridQueryTypeDescriptor type) throws IgniteCheckedException;
+    public void rebuildIndexesFromHash(String cacheName, GridQueryTypeDescriptor type) throws IgniteCheckedException;
 
     /**
      * Marks all indexes of given type for rebuild from hash index, making them unusable until rebuild finishes.
      *
-     * @param spaceName Space name.
+     * @param cacheName Cache name.
      * @param type Type descriptor.
      */
-    public void markForRebuildFromHash(String spaceName, GridQueryTypeDescriptor type);
+    public void markForRebuildFromHash(String cacheName, GridQueryTypeDescriptor type);
 
     /**
      * Returns backup filter.
@@ -272,19 +261,19 @@ public interface GridQueryIndexing {
     /**
      * Prepare native statement to retrieve JDBC metadata from.
      *
-     * @param space Schema.
+     * @param cacheName Cache name.
      * @param sql Query.
      * @return {@link PreparedStatement} from underlying engine to supply metadata to Prepared - most likely H2.
      */
-    public PreparedStatement prepareNativeStatement(String space, String sql) throws SQLException;
+    public PreparedStatement prepareNativeStatement(String cacheName, String sql) throws SQLException;
 
     /**
-     * Gets space name from database schema.
+     * Gets cache name from database schema.
      *
      * @param schemaName Schema name. Could not be null. Could be empty.
-     * @return Space name. Could be null.
+     * @return Cache name. Could be null.
      */
-    public String space(String schemaName);
+    public String cacheName(String schemaName);
 
     /**
      * Collect queries that already running more than specified duration.
@@ -307,7 +296,7 @@ public interface GridQueryIndexing {
     public void cancelAllQueries();
 
     /**
-     * @param spaceName Space name.
+     * @param cacheName Cache name.
      * @param nativeStmt Native statement.
      * @param autoFlushFreq Automatic data flushing frequency, disabled if {@code 0}.
      * @param nodeBufSize Per node buffer size - see {@link IgniteDataStreamer#perNodeBufferSize(int)}
@@ -316,6 +305,6 @@ public interface GridQueryIndexing {
      * @return {@link IgniteDataStreamer} tailored to specific needs of given native statement based on its metadata;
      * {@code null} if given statement is a query.
      */
-    public IgniteDataStreamer<?,?> createStreamer(String spaceName, PreparedStatement nativeStmt, long autoFlushFreq,
+    public IgniteDataStreamer<?,?> createStreamer(String cacheName, PreparedStatement nativeStmt, long autoFlushFreq,
         int nodeBufSize, int nodeParOps, boolean allowOverwrite);
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbf0b2a5/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 0df0f52..ced78cf 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
@@ -166,7 +166,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     /** Coordinator node (initialized lazily). */
     private ClusterNode crd;
 
-    /** Registered spaces. */
+    /** Registered cache names. */
     private final Collection<String> cacheNames = Collections.newSetFromMap(new ConcurrentHashMap<String, Boolean>());
 
     /** ID history for index create/drop discovery messages. */
@@ -668,7 +668,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
         try {
             synchronized (stateMux) {
-                String space = cctx.name();
+                String cacheName = cctx.name();
 
                 // Prepare candidates.
                 List<Class<?>> mustDeserializeClss = new ArrayList<>();
@@ -679,7 +679,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
                 if (!F.isEmpty(qryEntities)) {
                     for (QueryEntity qryEntity : qryEntities) {
-                        QueryTypeCandidate cand = QueryUtils.typeForQueryEntity(space, cctx, qryEntity,
+                        QueryTypeCandidate cand = QueryUtils.typeForQueryEntity(cacheName, cctx, qryEntity,
                             mustDeserializeClss);
 
                         cands.add(cand);
@@ -696,14 +696,14 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                     QueryTypeDescriptorImpl oldDesc = tblTypMap.put(desc.tableName(), desc);
 
                     if (oldDesc != null)
-                        throw new IgniteException("Duplicate table name [cache=" + space +
+                        throw new IgniteException("Duplicate table name [cache=" + cacheName +
                             ", tblName=" + desc.tableName() + ", type1=" + desc.name() + ", type2=" + oldDesc.name() + ']');
 
                     for (String idxName : desc.indexes().keySet()) {
                         oldDesc = idxTypMap.put(idxName, desc);
 
                         if (oldDesc != null)
-                            throw new IgniteException("Duplicate index name [cache=" + space +
+                            throw new IgniteException("Duplicate index name [cache=" + cacheName +
                                 ", idxName=" + idxName + ", type1=" + desc.name() + ", type2=" + oldDesc.name() + ']');
                     }
                 }
@@ -755,7 +755,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                 }
 
                 // Ready to register at this point.
-                registerCache0(space, cctx, cands);
+                registerCache0(cacheName, cctx, cands);
 
                 // Warn about possible implicit deserialization.
                 if (!mustDeserializeClss.isEmpty()) {
@@ -1268,7 +1268,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     /**
      * Register cache in indexing SPI.
      *
-     * @param cacheName Space.
+     * @param cacheName Cache name.
      * @param cctx Cache context.
      * @param cands Candidates.
      * @throws IgniteCheckedException If failed.
@@ -1324,7 +1324,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * Unregister cache.<p>
      * Use with {@link #busyLock} where appropriate.
      *
-     * @param cacheName Space.
+     * @param cacheName Cache name.
      */
     public void onCacheStop0(String cacheName) {
         if (idx == null)
@@ -1337,7 +1337,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             while (it.hasNext()) {
                 Map.Entry<QueryTypeIdKey, QueryTypeDescriptorImpl> entry = it.next();
 
-                if (F.eq(cacheName, entry.getKey().space())) {
+                if (F.eq(cacheName, entry.getKey().cacheName())) {
                     it.remove();
 
                     typesByName.remove(new QueryTypeNameKey(cacheName, entry.getValue().name()));
@@ -1354,7 +1354,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
                 QueryIndexKey idxKey = idxEntry.getKey();
 
-                if (F.eq(cacheName, idxKey.space()))
+                if (F.eq(cacheName, idxKey.cacheName()))
                     idxIt.remove();
             }
 
@@ -1377,21 +1377,21 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /**
-     * Check whether provided key and value belongs to expected space and table.
+     * Check whether provided key and value belongs to expected cache and table.
      *
      * @param cctx Target cache context.
-     * @param expSpace Expected space.
+     * @param expCacheName Expected cache name.
      * @param expTblName Expected table name.
      * @param key Key.
      * @param val Value.
-     * @return {@code True} if this key-value pair belongs to expected space/table, {@code false} otherwise or
-     *     if space or table doesn't exist.
+     * @return {@code True} if this key-value pair belongs to expected cache/table, {@code false} otherwise or
+     *     if cache or table doesn't exist.
      * @throws IgniteCheckedException If failed.
      */
     @SuppressWarnings("ConstantConditions")
-    public boolean belongsToTable(GridCacheContext cctx, String expSpace, String expTblName, KeyCacheObject key,
+    public boolean belongsToTable(GridCacheContext cctx, String expCacheName, String expTblName, KeyCacheObject key,
         CacheObject val) throws IgniteCheckedException {
-        QueryTypeDescriptorImpl desc = type(expSpace, val);
+        QueryTypeDescriptorImpl desc = type(expCacheName, val);
 
         if (desc == null)
             return false;
@@ -1424,14 +1424,14 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      */
     public IgniteInternalFuture<?> rebuildIndexesFromHash(Collection<Integer> cacheIds) {
         if (!busyLock.enterBusy())
-            throw new IllegalStateException("Failed to get space size (grid is stopping).");
+            throw new IllegalStateException("Failed to rebuild indexes from hash (grid is stopping).");
 
         try {
             GridCompoundFuture<Object, ?> fut = new GridCompoundFuture<Object, Object>();
 
             for (Map.Entry<QueryTypeIdKey, QueryTypeDescriptorImpl> e : types.entrySet()) {
-                if (cacheIds.contains(CU.cacheId(e.getKey().space())))
-                    fut.add(rebuildIndexesFromHash(e.getKey().space(), e.getValue()));
+                if (cacheIds.contains(CU.cacheId(e.getKey().cacheName())))
+                    fut.add(rebuildIndexesFromHash(e.getKey().cacheName(), e.getValue()));
             }
 
             fut.markInitialized();
@@ -1444,12 +1444,12 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /**
-     * @param space Space.
+     * @param cacheName Cache name.
      * @param desc Type descriptor.
      * @return Future that will be completed when rebuilding of all indexes is finished.
      */
     private IgniteInternalFuture<Object> rebuildIndexesFromHash(
-        @Nullable final String space,
+        @Nullable final String cacheName,
         @Nullable final QueryTypeDescriptorImpl desc
     ) {
         if (idx == null)
@@ -1460,12 +1460,12 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
         final GridWorkerFuture<Object> fut = new GridWorkerFuture<>();
 
-        idx.markForRebuildFromHash(space, desc);
+        idx.markForRebuildFromHash(cacheName, desc);
 
         GridWorker w = new GridWorker(ctx.igniteInstanceName(), "index-rebuild-worker", log) {
             @Override protected void body() {
                 try {
-                    idx.rebuildIndexesFromHash(space, desc);
+                    idx.rebuildIndexesFromHash(cacheName, desc);
 
                     fut.onDone();
                 }
@@ -1490,17 +1490,17 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /**
-     * @param space Space name.
+     * @param cacheName Cache name.
      * @return Cache object context.
      */
-    private CacheObjectContext cacheObjectContext(String space) {
-        return ctx.cache().internalCache(space).context().cacheObjectContext();
+    private CacheObjectContext cacheObjectContext(String cacheName) {
+        return ctx.cache().internalCache(cacheName).context().cacheObjectContext();
     }
 
     /**
      * Writes key-value pair to index.
      *
-     * @param space Space.
+     * @param cacheName Cache name.
      * @param key Key.
      * @param val Value.
      * @param ver Cache entry version.
@@ -1508,7 +1508,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @throws IgniteCheckedException In case of error.
      */
     @SuppressWarnings({"unchecked", "ConstantConditions"})
-    public void store(final String space,
+    public void store(final String cacheName,
         final KeyCacheObject key,
         int partId,
         @Nullable CacheObject prevVal,
@@ -1521,7 +1521,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         assert val != null;
 
         if (log.isDebugEnabled())
-            log.debug("Store [space=" + space + ", key=" + key + ", val=" + val + "]");
+            log.debug("Store [cache=" + cacheName + ", key=" + key + ", val=" + val + "]");
 
         if (idx == null)
             return;
@@ -1530,7 +1530,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             throw new NodeStoppingException("Operation has been cancelled (node is stopping).");
 
         try {
-            CacheObjectContext coctx = cacheObjectContext(space);
+            CacheObjectContext coctx = cacheObjectContext(cacheName);
 
             QueryTypeDescriptorImpl desc = typeByValue(coctx, key, val, true);
 
@@ -1538,13 +1538,13 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                 QueryTypeDescriptorImpl prevValDesc = typeByValue(coctx, key, prevVal, false);
 
                 if (prevValDesc != null && prevValDesc != desc)
-                    idx.remove(space, prevValDesc, key, partId, prevVal, prevVer);
+                    idx.remove(cacheName, prevValDesc, key, partId, prevVal, prevVer);
             }
 
             if (desc == null)
                 return;
 
-            idx.store(space, desc.name(), key, partId, val, ver, expirationTime, link);
+            idx.store(cacheName, desc.name(), key, partId, val, ver, expirationTime, link);
         }
         finally {
             busyLock.leaveBusy();
@@ -1606,25 +1606,25 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /**
-     * Gets type descriptor for space by given object's type.
+     * Gets type descriptor for cache by given object's type.
      *
-     * @param space Space name.
+     * @param cacheName Cache name.
      * @param val Object to determine type for.
      * @return Type descriptor.
      * @throws IgniteCheckedException If failed.
      */
     @SuppressWarnings("ConstantConditions")
-    private QueryTypeDescriptorImpl type(@Nullable String space, CacheObject val) throws IgniteCheckedException {
-        CacheObjectContext coctx = cacheObjectContext(space);
+    private QueryTypeDescriptorImpl type(@Nullable String cacheName, CacheObject val) throws IgniteCheckedException {
+        CacheObjectContext coctx = cacheObjectContext(cacheName);
 
         QueryTypeIdKey id;
 
         boolean binaryVal = ctx.cacheObjects().isBinaryObject(val);
 
         if (binaryVal)
-            id = new QueryTypeIdKey(space, ctx.cacheObjects().typeId(val));
+            id = new QueryTypeIdKey(cacheName, ctx.cacheObjects().typeId(val));
         else
-            id = new QueryTypeIdKey(space, val.value(coctx, false).getClass());
+            id = new QueryTypeIdKey(cacheName, val.value(coctx, false).getClass());
 
         return types.get(id);
     }
@@ -1718,12 +1718,12 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /**
-     * @param spaceName Cache name.
+     * @param cacheName Cache name.
      * @param streamer Data streamer.
      * @param qry Query.
      * @return Iterator.
      */
-    public long streamUpdateQuery(@Nullable final String spaceName,
+    public long streamUpdateQuery(@Nullable final String cacheName,
         final IgniteDataStreamer<?, ?> streamer, final String qry, final Object[] args) {
         assert streamer != null;
 
@@ -1731,11 +1731,11 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             throw new IllegalStateException("Failed to execute query (grid is stopping).");
 
         try {
-            GridCacheContext cctx = ctx.cache().cache(spaceName).context();
+            GridCacheContext cctx = ctx.cache().cache(cacheName).context();
 
             return executeQuery(GridCacheQueryType.SQL_FIELDS, qry, cctx, new IgniteOutClosureX<Long>() {
                 @Override public Long applyx() throws IgniteCheckedException {
-                    return idx.streamUpdateQuery(spaceName, qry, args, streamer);
+                    return idx.streamUpdateQuery(cacheName, qry, args, streamer);
                 }
             }, true);
         }
@@ -1966,28 +1966,28 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
     /**
      *
-     * @param space Space name.
+     * @param cacheName Cache name.
      * @param sql Query.
      * @return {@link PreparedStatement} from underlying engine to supply metadata to Prepared - most likely H2.
      */
-    public PreparedStatement prepareNativeStatement(String space, String sql) throws SQLException {
+    public PreparedStatement prepareNativeStatement(String cacheName, String sql) throws SQLException {
         checkxEnabled();
 
-        return idx.prepareNativeStatement(space, sql);
+        return idx.prepareNativeStatement(cacheName, sql);
     }
 
     /**
      * @param schema Schema name.
-     * @return space (cache) name from schema name.
+     * @return Cache name from schema name.
      */
-    public String space(String schema) throws SQLException {
+    public String cacheName(String schema) throws SQLException {
         checkxEnabled();
 
-        return idx.space(schema);
+        return idx.cacheName(schema);
     }
 
     /**
-     * @param spaceName Space name.
+     * @param cacheName Cache name.
      * @param nativeStmt Native statement.
      * @param autoFlushFreq Automatic data flushing frequency, disabled if {@code 0}.
      * @param nodeBufSize Per node buffer size - see {@link IgniteDataStreamer#perNodeBufferSize(int)}
@@ -1996,21 +1996,22 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @see IgniteDataStreamer#allowOverwrite
      * @return {@link IgniteDataStreamer} tailored to specific needs of given native statement based on its metadata.
      */
-    public IgniteDataStreamer<?, ?> createStreamer(String spaceName, PreparedStatement nativeStmt, long autoFlushFreq,
+    public IgniteDataStreamer<?, ?> createStreamer(String cacheName, PreparedStatement nativeStmt, long autoFlushFreq,
         int nodeBufSize, int nodeParOps, boolean allowOverwrite) {
-        return idx.createStreamer(spaceName, nativeStmt, autoFlushFreq, nodeBufSize, nodeParOps, allowOverwrite);
+        return idx.createStreamer(cacheName, nativeStmt, autoFlushFreq, nodeBufSize, nodeParOps, allowOverwrite);
     }
 
     /**
-     * @param space Space.
+     * @param cacheName Cache name.
      * @param key Key.
      * @throws IgniteCheckedException Thrown in case of any errors.
      */
-    public void remove(String space, KeyCacheObject key, int partId, CacheObject val, GridCacheVersion ver) throws IgniteCheckedException {
+    public void remove(String cacheName, KeyCacheObject key, int partId, CacheObject val, GridCacheVersion ver)
+        throws IgniteCheckedException {
         assert key != null;
 
         if (log.isDebugEnabled())
-            log.debug("Remove [space=" + space + ", key=" + key + ", val=" + val + "]");
+            log.debug("Remove [cacheName=" + cacheName + ", key=" + key + ", val=" + val + "]");
 
         if (idx == null)
             return;
@@ -2019,14 +2020,14 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             throw new IllegalStateException("Failed to remove from index (grid is stopping).");
 
         try {
-            CacheObjectContext coctx = cacheObjectContext(space);
+            CacheObjectContext coctx = cacheObjectContext(cacheName);
 
             QueryTypeDescriptorImpl desc = typeByValue(coctx, key, val, false);
 
             if (desc == null)
                 return;
 
-            idx.remove(space, desc, key, partId, val, ver);
+            idx.remove(cacheName, desc, key, partId, val, ver);
         }
         finally {
             busyLock.leaveBusy();
@@ -2034,7 +2035,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /**
-     * @param space Space.
+     * @param cacheName Cache name.
      * @param clause Clause.
      * @param resType Result type.
      * @param filters Key and value filters.
@@ -2044,7 +2045,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @throws IgniteCheckedException If failed.
      */
     @SuppressWarnings("unchecked")
-    public <K, V> GridCloseableIterator<IgniteBiTuple<K, V>> queryText(final String space, final String clause,
+    public <K, V> GridCloseableIterator<IgniteBiTuple<K, V>> queryText(final String cacheName, final String clause,
         final String resType, final IndexingQueryFilter filters) throws IgniteCheckedException {
         checkEnabled();
 
@@ -2052,14 +2053,14 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             throw new IllegalStateException("Failed to execute query (grid is stopping).");
 
         try {
-            final GridCacheContext<?, ?> cctx = ctx.cache().internalCache(space).context();
+            final GridCacheContext<?, ?> cctx = ctx.cache().internalCache(cacheName).context();
 
             return executeQuery(GridCacheQueryType.TEXT, clause, cctx,
                 new IgniteOutClosureX<GridCloseableIterator<IgniteBiTuple<K, V>>>() {
                     @Override public GridCloseableIterator<IgniteBiTuple<K, V>> applyx() throws IgniteCheckedException {
-                        String typeName = typeName(space, resType);
+                        String typeName = typeName(cacheName, resType);
 
-                        return idx.queryLocalText(space, clause, typeName, filters);
+                        return idx.queryLocalText(cacheName, clause, typeName, filters);
                     }
                 }, true);
         }
@@ -2069,33 +2070,33 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /**
-     * Gets types for space.
+     * Gets types for cache.
      *
-     * @param space Space name.
+     * @param cacheName Cache name.
      * @return Descriptors.
      */
-    public Collection<GridQueryTypeDescriptor> types(@Nullable String space) {
-        Collection<GridQueryTypeDescriptor> spaceTypes = new ArrayList<>();
+    public Collection<GridQueryTypeDescriptor> types(@Nullable String cacheName) {
+        Collection<GridQueryTypeDescriptor> cacheTypes = new ArrayList<>();
 
         for (Map.Entry<QueryTypeIdKey, QueryTypeDescriptorImpl> e : types.entrySet()) {
             QueryTypeDescriptorImpl desc = e.getValue();
 
-            if (F.eq(e.getKey().space(), space))
-                spaceTypes.add(desc);
+            if (F.eq(e.getKey().cacheName(), cacheName))
+                cacheTypes.add(desc);
         }
 
-        return spaceTypes;
+        return cacheTypes;
     }
 
     /**
-     * Get type descriptor for the given space and table name.
-     * @param space Space.
+     * Get type descriptor for the given cache and table name.
+     * @param cacheName Cache name.
      * @param tblName Table name.
      * @return Type (if any).
      */
-    @Nullable private QueryTypeDescriptorImpl type(@Nullable String space, String tblName) {
+    @Nullable private QueryTypeDescriptorImpl type(@Nullable String cacheName, String tblName) {
         for (QueryTypeDescriptorImpl type : types.values()) {
-            if (F.eq(space, type.space()) && F.eq(tblName, type.tableName()))
+            if (F.eq(cacheName, type.cacheName()) && F.eq(tblName, type.tableName()))
                 return type;
         }
 
@@ -2103,15 +2104,15 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /**
-     * Gets type name for provided space and type name if type is still valid.
+     * Gets type name for provided cache name and type name if type is still valid.
      *
-     * @param space Space name.
+     * @param cacheName Cache name.
      * @param typeName Type name.
      * @return Type descriptor.
      * @throws IgniteCheckedException If failed.
      */
-    private String typeName(@Nullable String space, String typeName) throws IgniteCheckedException {
-        QueryTypeDescriptorImpl type = typesByName.get(new QueryTypeNameKey(space, typeName));
+    private String typeName(@Nullable String cacheName, String typeName) throws IgniteCheckedException {
+        QueryTypeDescriptorImpl type = typesByName.get(new QueryTypeNameKey(cacheName, typeName));
 
         if (type == null)
             throw new IgniteCheckedException("Failed to find SQL table for type: " + typeName);

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbf0b2a5/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryIndexKey.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryIndexKey.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryIndexKey.java
index f580111..7fdb805 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryIndexKey.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryIndexKey.java
@@ -29,8 +29,8 @@ public class QueryIndexKey implements Serializable {
     /** */
     private static final long serialVersionUID = 0L;
 
-    /** Space. */
-    private final String space;
+    /** Cache name. */
+    private final String cacheName;
 
     /** Name. */
     private final String name;
@@ -38,19 +38,19 @@ public class QueryIndexKey implements Serializable {
     /**
      * Constructor.
      *
-     * @param space Space.
+     * @param cacheName Cache name.
      * @param name Name.
      */
-    public QueryIndexKey(String space, String name) {
-        this.space = space;
+    public QueryIndexKey(String cacheName, String name) {
+        this.cacheName = cacheName;
         this.name = name;
     }
 
     /**
-     * @return Space.
+     * @return Cache name.
      */
-    public String space() {
-        return space;
+    public String cacheName() {
+        return cacheName;
     }
 
     /**
@@ -62,7 +62,7 @@ public class QueryIndexKey implements Serializable {
 
     /** {@inheritDoc} */
     @Override public int hashCode() {
-        return 31 * (space != null ? space.hashCode() : 0) + (name != null ? name.hashCode() : 0);
+        return 31 * (cacheName != null ? cacheName.hashCode() : 0) + (name != null ? name.hashCode() : 0);
     }
 
     /** {@inheritDoc} */
@@ -75,7 +75,7 @@ public class QueryIndexKey implements Serializable {
 
         QueryIndexKey other = (QueryIndexKey)o;
 
-        return F.eq(name, other.name) && F.eq(space, other.space);
+        return F.eq(name, other.name) && F.eq(cacheName, other.cacheName);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbf0b2a5/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeDescriptorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeDescriptorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeDescriptorImpl.java
index 56c6aa5..4848b0a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeDescriptorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeDescriptorImpl.java
@@ -36,8 +36,8 @@ import java.util.Map;
  * Descriptor of type.
  */
 public class QueryTypeDescriptorImpl implements GridQueryTypeDescriptor {
-    /** Space. */
-    private final String space;
+    /** Cache name. */
+    private final String cacheName;
 
     /** */
     private String name;
@@ -102,17 +102,17 @@ public class QueryTypeDescriptorImpl implements GridQueryTypeDescriptor {
     /**
      * Constructor.
      *
-     * @param space Cache name.
+     * @param cacheName Cache name.
      */
-    public QueryTypeDescriptorImpl(String space) {
-        this.space = space;
+    public QueryTypeDescriptorImpl(String cacheName) {
+        this.cacheName = cacheName;
     }
 
     /**
-     * @return Space.
+     * @return Cache name.
      */
-    public String space() {
-        return space;
+    public String cacheName() {
+        return cacheName;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbf0b2a5/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeIdKey.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeIdKey.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeIdKey.java
index 4d486f9..fe7c487 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeIdKey.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeIdKey.java
@@ -20,11 +20,11 @@ package org.apache.ignite.internal.processors.query;
 import org.apache.ignite.internal.util.typedef.internal.S;
 
 /**
- * Identifying TypeDescriptor by space and value class.
+ * Identifying TypeDescriptor by cache name and value class.
  */
 public class QueryTypeIdKey {
     /** */
-    private final String space;
+    private final String cacheName;
 
     /** Value type. */
     private final Class<?> valType;
@@ -35,13 +35,13 @@ public class QueryTypeIdKey {
     /**
      * Constructor.
      *
-     * @param space Space name.
+     * @param cacheName Cache name.
      * @param valType Value type.
      */
-    public  QueryTypeIdKey(String space, Class<?> valType) {
+    public  QueryTypeIdKey(String cacheName, Class<?> valType) {
         assert valType != null;
 
-        this.space = space;
+        this.cacheName = cacheName;
         this.valType = valType;
 
         valTypeId = 0;
@@ -50,21 +50,21 @@ public class QueryTypeIdKey {
     /**
      * Constructor.
      *
-     * @param space Space name.
+     * @param cacheName Cache name.
      * @param valTypeId Value type ID.
      */
-    public QueryTypeIdKey(String space, int valTypeId) {
-        this.space = space;
+    public QueryTypeIdKey(String cacheName, int valTypeId) {
+        this.cacheName = cacheName;
         this.valTypeId = valTypeId;
 
         valType = null;
     }
 
     /**
-     * @return Space.
+     * @return Cache name.
      */
-    public String space() {
-        return space;
+    public String cacheName() {
+        return cacheName;
     }
 
     /** {@inheritDoc} */
@@ -79,12 +79,12 @@ public class QueryTypeIdKey {
 
         return (valTypeId == typeId.valTypeId) &&
             (valType != null ? valType == typeId.valType : typeId.valType == null) &&
-            (space != null ? space.equals(typeId.space) : typeId.space == null);
+            (cacheName != null ? cacheName.equals(typeId.cacheName) : typeId.cacheName == null);
     }
 
     /** {@inheritDoc} */
     @Override public int hashCode() {
-        return 31 * (space != null ? space.hashCode() : 0) + (valType != null ? valType.hashCode() : valTypeId);
+        return 31 * (cacheName != null ? cacheName.hashCode() : 0) + (valType != null ? valType.hashCode() : valTypeId);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbf0b2a5/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeNameKey.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeNameKey.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeNameKey.java
index 8a36a03..b10c5b2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeNameKey.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeNameKey.java
@@ -26,19 +26,19 @@ import org.jetbrains.annotations.Nullable;
  */
 public class QueryTypeNameKey {
     /** */
-    private final String space;
+    private final String cacheName;
 
     /** */
     private final String typeName;
 
     /**
-     * @param space Space name.
+     * @param cacheName Cache name.
      * @param typeName Type name.
      */
-    public QueryTypeNameKey(@Nullable String space, String typeName) {
+    public QueryTypeNameKey(@Nullable String cacheName, String typeName) {
         assert !F.isEmpty(typeName) : typeName;
 
-        this.space = space;
+        this.cacheName = cacheName;
         this.typeName = typeName;
     }
 
@@ -52,13 +52,13 @@ public class QueryTypeNameKey {
 
         QueryTypeNameKey other = (QueryTypeNameKey)o;
 
-        return (space != null ? space.equals(other.space) : other.space == null) &&
+        return (cacheName != null ? cacheName.equals(other.cacheName) : other.cacheName == null) &&
             typeName.equals(other.typeName);
     }
 
     /** {@inheritDoc} */
     @Override public int hashCode() {
-        return 31 * (space != null ? space.hashCode() : 0) + typeName.hashCode();
+        return 31 * (cacheName != null ? cacheName.hashCode() : 0) + typeName.hashCode();
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbf0b2a5/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
index 1a80a37..245965c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
@@ -163,14 +163,14 @@ public class QueryUtils {
     /**
      * Create type candidate for query entity.
      *
-     * @param space Space.
+     * @param cacheName Cache name.
      * @param cctx Cache context.
      * @param qryEntity Query entity.
      * @param mustDeserializeClss Classes which must be deserialized.
      * @return Type candidate.
      * @throws IgniteCheckedException If failed.
      */
-    public static QueryTypeCandidate typeForQueryEntity(String space, GridCacheContext cctx, QueryEntity qryEntity,
+    public static QueryTypeCandidate typeForQueryEntity(String cacheName, GridCacheContext cctx, QueryEntity qryEntity,
         List<Class<?>> mustDeserializeClss) throws IgniteCheckedException {
         GridKernalContext ctx = cctx.kernalContext();
         CacheConfiguration<?,?> ccfg = cctx.config();
@@ -179,7 +179,7 @@ public class QueryUtils {
 
         CacheObjectContext coCtx = binaryEnabled ? ctx.cacheObjects().contextForCache(ccfg) : null;
 
-        QueryTypeDescriptorImpl desc = new QueryTypeDescriptorImpl(space);
+        QueryTypeDescriptorImpl desc = new QueryTypeDescriptorImpl(cacheName);
 
         desc.aliases(qryEntity.getAliases());
 
@@ -244,10 +244,10 @@ public class QueryUtils {
         if (valCls == null || (binaryEnabled && !keyOrValMustDeserialize)) {
             processBinaryMeta(ctx, qryEntity, desc);
 
-            typeId = new QueryTypeIdKey(space, ctx.cacheObjects().typeId(qryEntity.findValueType()));
+            typeId = new QueryTypeIdKey(cacheName, ctx.cacheObjects().typeId(qryEntity.findValueType()));
 
             if (valCls != null)
-                altTypeId = new QueryTypeIdKey(space, valCls);
+                altTypeId = new QueryTypeIdKey(cacheName, valCls);
 
             if (!cctx.customAffinityMapper() && qryEntity.findKeyType() != null) {
                 // Need to setup affinity key for distributed joins.
@@ -270,8 +270,8 @@ public class QueryUtils {
                     desc.affinityKey(affField);
             }
 
-            typeId = new QueryTypeIdKey(space, valCls);
-            altTypeId = new QueryTypeIdKey(space, ctx.cacheObjects().typeId(qryEntity.findValueType()));
+            typeId = new QueryTypeIdKey(cacheName, valCls);
+            altTypeId = new QueryTypeIdKey(cacheName, ctx.cacheObjects().typeId(qryEntity.findValueType()));
         }
 
         return new QueryTypeCandidate(typeId, altTypeId, desc);

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbf0b2a5/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/SchemaIndexCacheVisitorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/SchemaIndexCacheVisitorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/SchemaIndexCacheVisitorImpl.java
index 58c909d..b3fa47c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/SchemaIndexCacheVisitorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/SchemaIndexCacheVisitorImpl.java
@@ -48,8 +48,8 @@ public class SchemaIndexCacheVisitorImpl implements SchemaIndexCacheVisitor {
     /** Cache context. */
     private final GridCacheContext cctx;
 
-    /** Space name. */
-    private final String spaceName;
+    /** Cache name. */
+    private final String cacheName;
 
     /** Table name. */
     private final String tblName;
@@ -61,14 +61,14 @@ public class SchemaIndexCacheVisitorImpl implements SchemaIndexCacheVisitor {
      * Constructor.
      *
      * @param cctx Cache context.
-     * @param spaceName Space name.
+     * @param cacheName Cache name.
      * @param tblName Table name.
      * @param cancel Cancellation token.
      */
-    public SchemaIndexCacheVisitorImpl(GridQueryProcessor qryProc, GridCacheContext cctx, String spaceName,
+    public SchemaIndexCacheVisitorImpl(GridQueryProcessor qryProc, GridCacheContext cctx, String cacheName,
         String tblName, SchemaIndexOperationCancellationToken cancel) {
         this.qryProc = qryProc;
-        this.spaceName = spaceName;
+        this.cacheName = cacheName;
         this.tblName = tblName;
         this.cancel = cancel;
 
@@ -190,7 +190,7 @@ public class SchemaIndexCacheVisitorImpl implements SchemaIndexCacheVisitor {
         /** {@inheritDoc} */
         @Override public void apply(KeyCacheObject key, int part, CacheObject val, GridCacheVersion ver,
             long expiration, long link) throws IgniteCheckedException {
-            if (qryProc.belongsToTable(cctx, spaceName, tblName, key, val))
+            if (qryProc.belongsToTable(cctx, cacheName, tblName, key, val))
                 target.apply(key, part, val, ver, expiration, link);
         }
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbf0b2a5/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/operation/SchemaAbstractOperation.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/operation/SchemaAbstractOperation.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/operation/SchemaAbstractOperation.java
index 726c90c..c45e229 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/operation/SchemaAbstractOperation.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/operation/SchemaAbstractOperation.java
@@ -58,7 +58,7 @@ public abstract class SchemaAbstractOperation implements Serializable {
     }
 
     /**
-     * @return Space.
+     * @return Cache name.
      */
     public String cacheName() {
         return cacheName;

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbf0b2a5/modules/core/src/main/java/org/apache/ignite/spi/indexing/IndexingQueryFilter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/indexing/IndexingQueryFilter.java b/modules/core/src/main/java/org/apache/ignite/spi/indexing/IndexingQueryFilter.java
index 935feab..74d349a 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/indexing/IndexingQueryFilter.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/indexing/IndexingQueryFilter.java
@@ -25,17 +25,17 @@ import org.jetbrains.annotations.Nullable;
  */
 public interface IndexingQueryFilter {
     /**
-     * Creates optional predicate for space.
+     * Creates optional predicate for cache.
      *
-     * @param spaceName Space name.
+     * @param cacheName Cache name.
      * @return Predicate or {@code null} if no filtering is needed.
      */
-    @Nullable public <K, V> IgniteBiPredicate<K, V> forSpace(String spaceName);
+    @Nullable public <K, V> IgniteBiPredicate<K, V> forCache(String cacheName);
 
     /**
      * Is the value required for filtering logic?
      * If false then null instead of value will be passed
-     * to IgniteBiPredicate returned by {@link #forSpace(String)} method.
+     * to IgniteBiPredicate returned by {@link #forCache(String)} method.
      *
      * @return true if value is required for filtering, false otherwise.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbf0b2a5/modules/core/src/main/java/org/apache/ignite/spi/indexing/IndexingSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/indexing/IndexingSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/indexing/IndexingSpi.java
index 4d53bea..8ec4a67 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/indexing/IndexingSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/indexing/IndexingSpi.java
@@ -25,8 +25,7 @@ import org.apache.ignite.spi.IgniteSpiException;
 import org.jetbrains.annotations.Nullable;
 
 /**
- * Indexing SPI allows user to index cache content. Using indexing SPI user can index data in cache and run
- * Usually cache name will be used as space name, so multiple caches can write to single indexing SPI instance.
+ * Indexing SPI allows user to index cache content. Using indexing SPI user can index data in cache and run queries.
  * <p>
  * <b>NOTE:</b> this SPI (i.e. methods in this interface) should never be used directly. SPIs provide
  * internal view on the subsystem and is used internally by Ignite kernal. In rare use cases when
@@ -66,33 +65,33 @@ public interface IndexingSpi extends IgniteSpi {
     /**
      * Executes query.
      *
-     * @param spaceName Space name.
+     * @param cacheName Cache name.
      * @param params Query parameters.
      * @param filters System filters.
      * @return Query result. If the iterator implements {@link AutoCloseable} it will be correctly closed.
      * @throws IgniteSpiException If failed.
      */
-    public Iterator<Cache.Entry<?,?>> query(@Nullable String spaceName, Collection<Object> params,
+    public Iterator<Cache.Entry<?,?>> query(@Nullable String cacheName, Collection<Object> params,
         @Nullable IndexingQueryFilter filters) throws IgniteSpiException;
 
     /**
-     * Updates index. Note that key is unique for space, so if space contains multiple indexes
+     * Updates index. Note that key is unique for cache, so if cache contains multiple indexes
      * the key should be removed from indexes other than one being updated.
      *
-     * @param spaceName Space name.
+     * @param cacheName Cache name.
      * @param key Key.
      * @param val Value.
      * @param expirationTime Expiration time or 0 if never expires.
      * @throws IgniteSpiException If failed.
      */
-    public void store(@Nullable String spaceName, Object key, Object val, long expirationTime) throws IgniteSpiException;
+    public void store(@Nullable String cacheName, Object key, Object val, long expirationTime) throws IgniteSpiException;
 
     /**
      * Removes index entry by key.
      *
-     * @param spaceName Space name.
+     * @param cacheName Cache name.
      * @param key Key.
      * @throws IgniteSpiException If failed.
      */
-    public void remove(@Nullable String spaceName, Object key) throws IgniteSpiException;
+    public void remove(@Nullable String cacheName, Object key) throws IgniteSpiException;
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbf0b2a5/modules/core/src/main/java/org/apache/ignite/spi/indexing/noop/NoopIndexingSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/indexing/noop/NoopIndexingSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/indexing/noop/NoopIndexingSpi.java
index 0ed7e33..5677f55 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/indexing/noop/NoopIndexingSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/indexing/noop/NoopIndexingSpi.java
@@ -33,19 +33,19 @@ import org.jetbrains.annotations.Nullable;
 @IgniteSpiNoop
 public class NoopIndexingSpi extends IgniteSpiAdapter implements IndexingSpi {
     /** {@inheritDoc} */
-    @Override public Iterator<Cache.Entry<?,?>> query(@Nullable String spaceName, Collection<Object> params,
+    @Override public Iterator<Cache.Entry<?,?>> query(@Nullable String cacheName, Collection<Object> params,
         @Nullable IndexingQueryFilter filters) throws IgniteSpiException {
         throw new IgniteSpiException("You have to configure custom GridIndexingSpi implementation.");
     }
 
     /** {@inheritDoc} */
-    @Override public void store(@Nullable String spaceName, Object key, Object val, long expirationTime)
+    @Override public void store(@Nullable String cacheName, Object key, Object val, long expirationTime)
         throws IgniteSpiException {
         assert false;
     }
 
     /** {@inheritDoc} */
-    @Override public void remove(@Nullable String spaceName, Object key) throws IgniteSpiException {
+    @Override public void remove(@Nullable String cacheName, Object key) throws IgniteSpiException {
         assert false;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbf0b2a5/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxExceptionAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxExceptionAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxExceptionAbstractSelfTest.java
index 1d27524..ac294b0 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxExceptionAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxExceptionAbstractSelfTest.java
@@ -664,13 +664,13 @@ public abstract class IgniteTxExceptionAbstractSelfTest extends GridCacheAbstrac
         }
 
         /** {@inheritDoc} */
-        @Override public Iterator<Cache.Entry<?, ?>> query(@Nullable String spaceName, Collection<Object> params,
+        @Override public Iterator<Cache.Entry<?, ?>> query(@Nullable String cacheName, Collection<Object> params,
             @Nullable IndexingQueryFilter filters) throws IgniteSpiException {
             throw new UnsupportedOperationException();
         }
 
         /** {@inheritDoc} */
-        @Override public void store(@Nullable String spaceName, Object key, Object val, long expirationTime)
+        @Override public void store(@Nullable String cacheName, Object key, Object val, long expirationTime)
             throws IgniteSpiException {
             if (fail) {
                 fail = false;
@@ -680,7 +680,7 @@ public abstract class IgniteTxExceptionAbstractSelfTest extends GridCacheAbstrac
         }
 
         /** {@inheritDoc} */
-        @Override public void remove(@Nullable String spaceName, Object k)
+        @Override public void remove(@Nullable String cacheName, Object k)
             throws IgniteSpiException {
             if (fail) {
                 fail = false;

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbf0b2a5/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IndexingSpiQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IndexingSpiQuerySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IndexingSpiQuerySelfTest.java
index 7349a4e..b6e32d5 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IndexingSpiQuerySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IndexingSpiQuerySelfTest.java
@@ -254,7 +254,7 @@ public class IndexingSpiQuerySelfTest extends TestCase {
         }
 
         /** {@inheritDoc} */
-        @Override public Iterator<Cache.Entry<?, ?>> query(@Nullable String spaceName, Collection<Object> params,
+        @Override public Iterator<Cache.Entry<?, ?>> query(@Nullable String cacheName, Collection<Object> params,
             @Nullable IndexingQueryFilter filters) throws IgniteSpiException {
             if (params.size() < 2)
                 throw new IgniteSpiException("Range parameters required.");
@@ -278,7 +278,7 @@ public class IndexingSpiQuerySelfTest extends TestCase {
         }
 
         /** {@inheritDoc} */
-        @Override public void store(@Nullable String spaceName, Object key, Object val, long expirationTime)
+        @Override public void store(@Nullable String cacheName, Object key, Object val, long expirationTime)
             throws IgniteSpiException {
             assertFalse(key instanceof BinaryObject);
             assertFalse(val instanceof BinaryObject);
@@ -287,7 +287,7 @@ public class IndexingSpiQuerySelfTest extends TestCase {
         }
 
         /** {@inheritDoc} */
-        @Override public void remove(@Nullable String spaceName, Object key) throws IgniteSpiException {
+        @Override public void remove(@Nullable String cacheName, Object key) throws IgniteSpiException {
             // No-op.
         }
     }
@@ -298,17 +298,17 @@ public class IndexingSpiQuerySelfTest extends TestCase {
     private static class MyBinaryIndexingSpi extends MyIndexingSpi {
 
         /** {@inheritDoc} */
-        @Override public void store(@Nullable String spaceName, Object key, Object val,
+        @Override public void store(@Nullable String cacheName, Object key, Object val,
             long expirationTime) throws IgniteSpiException {
             assertTrue(key instanceof BinaryObject);
 
             assertTrue(val instanceof BinaryObject);
 
-            super.store(spaceName, ((BinaryObject)key).deserialize(), ((BinaryObject)val).deserialize(), expirationTime);
+            super.store(cacheName, ((BinaryObject)key).deserialize(), ((BinaryObject)val).deserialize(), expirationTime);
         }
 
         /** {@inheritDoc} */
-        @Override public void remove(@Nullable String spaceName, Object key) throws IgniteSpiException {
+        @Override public void remove(@Nullable String cacheName, Object key) throws IgniteSpiException {
             assertTrue(key instanceof BinaryObject);
         }
     }
@@ -318,7 +318,7 @@ public class IndexingSpiQuerySelfTest extends TestCase {
      */
     private static class MyBrokenIndexingSpi extends MyIndexingSpi {
         /** {@inheritDoc} */
-        @Override public void store(@Nullable String spaceName, Object key, Object val,
+        @Override public void store(@Nullable String cacheName, Object key, Object val,
             long expirationTime) throws IgniteSpiException {
             throw new IgniteSpiException("Test exception");
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbf0b2a5/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IndexingSpiQueryTxSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IndexingSpiQueryTxSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IndexingSpiQueryTxSelfTest.java
index 9d2b31c..e59deed 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IndexingSpiQueryTxSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IndexingSpiQueryTxSelfTest.java
@@ -133,19 +133,19 @@ public class IndexingSpiQueryTxSelfTest extends GridCacheAbstractSelfTest {
         }
 
         /** {@inheritDoc} */
-        @Override public Iterator<Cache.Entry<?, ?>> query(@Nullable String spaceName, Collection<Object> params,
+        @Override public Iterator<Cache.Entry<?, ?>> query(@Nullable String cacheName, Collection<Object> params,
             @Nullable IndexingQueryFilter filters) throws IgniteSpiException {
            return null;
         }
 
         /** {@inheritDoc} */
-        @Override public void store(@Nullable String spaceName, Object key, Object val, long expirationTime)
+        @Override public void store(@Nullable String cacheName, Object key, Object val, long expirationTime)
             throws IgniteSpiException {
             throw new IgniteSpiException("Test exception");
         }
 
         /** {@inheritDoc} */
-        @Override public void remove(@Nullable String spaceName, Object key) throws IgniteSpiException {
+        @Override public void remove(@Nullable String cacheName, Object key) throws IgniteSpiException {
             // No-op.
         }
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbf0b2a5/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
index 2a3d77c..db7bfd6 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
@@ -122,31 +122,31 @@ public class DmlStatementsProcessor {
     /**
      * Handle cache stop.
      *
-     * @param spaceName Cache name.
+     * @param cacheName Cache name.
      */
-    public void onCacheStop(String spaceName) {
-        planCache.remove(spaceName);
+    public void onCacheStop(String cacheName) {
+        planCache.remove(cacheName);
     }
 
     /**
      * Execute DML statement, possibly with few re-attempts in case of concurrent data modifications.
      *
-     * @param spaceName Space name.
+     * @param cacheName Cache name.
      * @param stmt JDBC statement.
      * @param fieldsQry Original query.
      * @param loc Query locality flag.
-     * @param filters Space name and key filter.
+     * @param filters Cache name and key filter.
      * @param cancel Cancel.
      * @return Update result (modified items count and failed keys).
      * @throws IgniteCheckedException if failed.
      */
-    private UpdateResult updateSqlFields(String spaceName, PreparedStatement stmt, SqlFieldsQuery fieldsQry,
+    private UpdateResult updateSqlFields(String cacheName, PreparedStatement stmt, SqlFieldsQuery fieldsQry,
         boolean loc, IndexingQueryFilter filters, GridQueryCancel cancel) throws IgniteCheckedException {
         Object[] errKeys = null;
 
         long items = 0;
 
-        UpdatePlan plan = getPlanForStatement(spaceName, stmt, null);
+        UpdatePlan plan = getPlanForStatement(cacheName, stmt, null);
 
         GridCacheContext<?, ?> cctx = plan.tbl.rowDescriptor().context();
 
@@ -194,7 +194,7 @@ public class DmlStatementsProcessor {
     }
 
     /**
-     * @param spaceName Space name.
+     * @param cacheName Cache name.
      * @param stmt Prepared statement.
      * @param fieldsQry Initial query.
      * @param cancel Query cancel.
@@ -202,9 +202,9 @@ public class DmlStatementsProcessor {
      * @throws IgniteCheckedException if failed.
      */
     @SuppressWarnings("unchecked")
-    QueryCursorImpl<List<?>> updateSqlFieldsTwoStep(String spaceName, PreparedStatement stmt,
+    QueryCursorImpl<List<?>> updateSqlFieldsTwoStep(String cacheName, PreparedStatement stmt,
         SqlFieldsQuery fieldsQry, GridQueryCancel cancel) throws IgniteCheckedException {
-        UpdateResult res = updateSqlFields(spaceName, stmt, fieldsQry, false, null, cancel);
+        UpdateResult res = updateSqlFields(cacheName, stmt, fieldsQry, false, null, cancel);
 
         QueryCursorImpl<List<?>> resCur = (QueryCursorImpl<List<?>>)new QueryCursorImpl(Collections.singletonList
             (Collections.singletonList(res.cnt)), null, false);
@@ -216,17 +216,17 @@ public class DmlStatementsProcessor {
 
     /**
      * Execute DML statement on local cache.
-     * @param spaceName Space name.
+     * @param cacheName Cache name.
      * @param stmt Prepared statement.
-     * @param filters Space name and key filter.
+     * @param filters Cache name and key filter.
      * @param cancel Query cancel.
      * @return Update result wrapped into {@link GridQueryFieldsResult}
      * @throws IgniteCheckedException if failed.
      */
     @SuppressWarnings("unchecked")
-    GridQueryFieldsResult updateLocalSqlFields(String spaceName, PreparedStatement stmt,
+    GridQueryFieldsResult updateLocalSqlFields(String cacheName, PreparedStatement stmt,
         SqlFieldsQuery fieldsQry, IndexingQueryFilter filters, GridQueryCancel cancel) throws IgniteCheckedException {
-        UpdateResult res = updateSqlFields(spaceName, stmt, fieldsQry, true, filters, cancel);
+        UpdateResult res = updateSqlFields(cacheName, stmt, fieldsQry, true, filters, cancel);
 
         return new GridQueryFieldsResultAdapter(UPDATE_RESULT_META,
             new IgniteSingletonIterator(Collections.singletonList(res.cnt)));
@@ -317,8 +317,9 @@ public class DmlStatementsProcessor {
      * Actually perform SQL DML operation locally.
      * @param cctx Cache context.
      * @param prepStmt Prepared statement for DML query.
-     * @param filters Space name and key filter.
-     * @param failedKeys Keys to restrict UPDATE and DELETE operations with. Null or empty array means no restriction.   @return Pair [number of successfully processed items; keys that have failed to be processed]
+     * @param filters Cache name and key filter.
+     * @param failedKeys Keys to restrict UPDATE and DELETE operations with. Null or empty array means no restriction.
+     * @return Pair [number of successfully processed items; keys that have failed to be processed]
      * @throws IgniteCheckedException if failed.
      */
     @SuppressWarnings({"ConstantConditions", "unchecked"})
@@ -392,28 +393,28 @@ public class DmlStatementsProcessor {
     /**
      * Generate SELECT statements to retrieve data for modifications from and find fast UPDATE or DELETE args,
      * if available.
-     * @param spaceName Space name.
+     * @param cacheName Cache name.
      * @param prepStmt JDBC statement.
      * @return Update plan.
      */
     @SuppressWarnings({"unchecked", "ConstantConditions"})
-    private UpdatePlan getPlanForStatement(String spaceName, PreparedStatement prepStmt,
+    private UpdatePlan getPlanForStatement(String cacheName, PreparedStatement prepStmt,
         @Nullable Integer errKeysPos) throws IgniteCheckedException {
         Prepared p = GridSqlQueryParser.prepared(prepStmt);
 
-        spaceName = F.isEmpty(spaceName) ? "default" : spaceName;
+        cacheName = F.isEmpty(cacheName) ? "default" : cacheName;
 
-        ConcurrentMap<String, UpdatePlan> spacePlans = planCache.get(spaceName);
+        ConcurrentMap<String, UpdatePlan> cachePlans = planCache.get(cacheName);
 
-        if (spacePlans == null) {
-            spacePlans = new GridBoundedConcurrentLinkedHashMap<>(PLAN_CACHE_SIZE);
+        if (cachePlans == null) {
+            cachePlans = new GridBoundedConcurrentLinkedHashMap<>(PLAN_CACHE_SIZE);
 
-            spacePlans = U.firstNotNull(planCache.putIfAbsent(spaceName, spacePlans), spacePlans);
+            cachePlans = U.firstNotNull(planCache.putIfAbsent(cacheName, cachePlans), cachePlans);
         }
 
         // getSQL returns field value, so it's fast
         // Don't look for re-runs in cache, we don't cache them
-        UpdatePlan res = (errKeysPos == null ? spacePlans.get(p.getSQL()) : null);
+        UpdatePlan res = (errKeysPos == null ? cachePlans.get(p.getSQL()) : null);
 
         if (res != null)
             return res;
@@ -422,7 +423,7 @@ public class DmlStatementsProcessor {
 
         // Don't cache re-runs
         if (errKeysPos == null)
-            return U.firstNotNull(spacePlans.putIfAbsent(p.getSQL(), res), res);
+            return U.firstNotNull(cachePlans.putIfAbsent(p.getSQL(), res), res);
         else
             return res;
     }