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 8da385a58e3f0..f6f79fbb0fd14 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 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 @@ private Ignite getIgnite(String cfgUrl) throws IgniteCheckedException { @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} */ 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 a60cdbd0b3d82..6d2960437f60d 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 GridIndexingManager(GridKernalContext ctx) { /** * 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 void store(final String space, final K key, final V val, long expirationTime) throws IgniteCheckedException { + public 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 void store(final String space, final K key, final V val, long expi 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 void store(final String space, final K key, final V val, long expi } /** - * @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 void remove(String space, Object key) throws IgniteCheckedException { 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 void remove(String space, Object key) throws IgniteCheckedException { } /** - * @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 params, IndexingQueryFilter filters) + public IgniteSpiCloseableIterator query(String cacheName, Collection params, IndexingQueryFilter filters) throws IgniteCheckedException { if (!enabled()) throw new IgniteCheckedException("Indexing SPI is not configured."); @@ -143,7 +144,7 @@ public IgniteSpiCloseableIterator query(String space, Collection para 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<>(); 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 0a789ae615932..07545a588afc1 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 extends GridCacheManagerAdapte private GridQueryProcessor qryProc; /** */ - private String space; + private String cacheName; /** */ private int maxIterCnt; @@ -221,9 +221,9 @@ public abstract class GridCacheQueryManager 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 void store(KeyCacheObject key, 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 void remove(KeyCacheObject key, int partId, CacheObject val, GridCacheVer 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 @@ private QueryResult executeQuery(GridCacheQueryAdapter qry, 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 @@ private FieldsResult executeFieldsQuery(GridCacheQueryAdapter qry, @Nullable 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 Collection sqlMetadata() throws IgniteCheckedExcept // Remote nodes that have current cache. Collection nodes = F.view(cctx.discovery().remoteNodes(), new P1() { @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 Collection sqlMetadata() throws IgniteCheckedExcept Collection 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 metas : map.values()) col.add(new CacheSqlMetadata(metas)); @@ -1969,10 +1969,10 @@ public Collection sqlMetadata() throws IgniteCheckedExcept return null; return new IndexingQueryFilter() { - @Nullable @Override public IgniteBiPredicate forSpace(final String spaceName) { + @Nullable @Override public IgniteBiPredicate forCache(final String cacheName) { final GridKernalContext ctx = cctx.kernalContext(); - final GridCacheAdapter cache = ctx.cache().internalCache(spaceName); + final GridCacheAdapter cache = ctx.cache().internalCache(cacheName); if (cache.context().isReplicated() || cache.configuration().getBackups() == 0) return null; @@ -2016,10 +2016,10 @@ private IndexingQueryFilter filter(GridCacheQueryAdapter qry) { /** * 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; } /** 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 bfa223efdd80e..1f5e701e2ebae 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 FieldsQueryCursor> queryDistributedSqlFields(GridCacheContext streamer) throws IgniteCheckedException; /** @@ -104,7 +104,7 @@ public long streamUpdateQuery(final String spaceName, final String qry, @Nullabl * * @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 QueryCursor> queryLocalSql(GridCacheContext * * @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 FieldsQueryCursor> queryLocalSqlFields(GridCacheContext cct /** * 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 GridCloseableIterator> queryLocalText(String spaceName, String qry, + public GridCloseableIterator> 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 void registerCache(String spaceName, GridCacheContext cctx, CacheCon * @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 void rebuildIndexesFromHash(String spaceName, /** * 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 void rebuildIndexesFromHash(String spaceName, 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 void rebuildIndexesFromHash(String spaceName, * @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); } 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 0df0f529206ec..ced78cfb2e3f1 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 cacheNames = Collections.newSetFromMap(new ConcurrentHashMap()); /** ID history for index create/drop discovery messages. */ @@ -668,7 +668,7 @@ public void onCacheStart0(GridCacheContext cctx, QuerySchema schema) try { synchronized (stateMux) { - String space = cctx.name(); + String cacheName = cctx.name(); // Prepare candidates. List> mustDeserializeClss = new ArrayList<>(); @@ -679,7 +679,7 @@ public void onCacheStart0(GridCacheContext cctx, QuerySchema schema) 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 void onCacheStart0(GridCacheContext cctx, QuerySchema schema) 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 @@ else if (op0 instanceof SchemaIndexDropOperation) { } // 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 @@ else if (op instanceof SchemaIndexDropOperation) { /** * 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 @@ private void registerCache0(String cacheName, GridCacheContext cctx, Colle * Unregister cache.

* 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 void onCacheStop0(String cacheName) { while (it.hasNext()) { Map.Entry 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 void onCacheStop0(String cacheName) { QueryIndexKey idxKey = idxEntry.getKey(); - if (F.eq(cacheName, idxKey.space())) + if (F.eq(cacheName, idxKey.cacheName())) idxIt.remove(); } @@ -1377,21 +1377,21 @@ public void onCacheStop0(String cacheName) { } /** - * 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 boolean belongsToTable(GridCacheContext cctx, String expSpace, String exp */ public IgniteInternalFuture rebuildIndexesFromHash(Collection 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 fut = new GridCompoundFuture(); for (Map.Entry 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 IgniteInternalFuture rebuildIndexesFromHash(Collection cacheI } /** - * @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 rebuildIndexesFromHash( - @Nullable final String space, + @Nullable final String cacheName, @Nullable final QueryTypeDescriptorImpl desc ) { if (idx == null) @@ -1460,12 +1460,12 @@ private IgniteInternalFuture rebuildIndexesFromHash( final GridWorkerFuture 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 @@ private IgniteInternalFuture rebuildIndexesFromHash( } /** - * @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 @@ private CacheObjectContext cacheObjectContext(String space) { * @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 void store(final String space, 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 void store(final String space, 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 void store(final String space, 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 void store(final String space, } /** - * 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 FieldsQueryCursor> querySqlFields(final GridCacheContext cct } /** - * @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 long streamUpdateQuery(@Nullable final String spaceName, 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() { @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 @@ private void sendQueryExecutedEvent(String sqlQry, Object[] params, String cache /** * - * @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 String space(String schema) throws SQLException { * @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 void remove(String space, KeyCacheObject key, int partId, CacheObject val 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 void remove(String space, KeyCacheObject key, int partId, CacheObject val } /** - * @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 void remove(String space, KeyCacheObject key, int partId, CacheObject val * @throws IgniteCheckedException If failed. */ @SuppressWarnings("unchecked") - public GridCloseableIterator> queryText(final String space, final String clause, + public GridCloseableIterator> queryText(final String cacheName, final String clause, final String resType, final IndexingQueryFilter filters) throws IgniteCheckedException { checkEnabled(); @@ -2052,14 +2053,14 @@ public GridCloseableIterator> queryText(final String 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>>() { @Override public GridCloseableIterator> 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 GridCloseableIterator> queryText(final String } /** - * Gets types for space. + * Gets types for cache. * - * @param space Space name. + * @param cacheName Cache name. * @return Descriptors. */ - public Collection types(@Nullable String space) { - Collection spaceTypes = new ArrayList<>(); + public Collection types(@Nullable String cacheName) { + Collection cacheTypes = new ArrayList<>(); for (Map.Entry 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 Collection types(@Nullable String space) { } /** - * 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); 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 f5801110c4f9e..7fdb8056ec3ec 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 String name() { /** {@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 String name() { 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} */ 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 56c6aa5adad3e..4848b0a1e8a54 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 @@ * 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} */ 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 4d486f9a0195b..fe7c48733abe3 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 @@ 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 QueryTypeIdKey(String space, Class valType) { /** * 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 String space() { 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} */ 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 8a36a03abce29..b10c5b29d9d10 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 @@ */ 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 QueryTypeNameKey(@Nullable String space, String typeName) { 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} */ 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 1a80a370a373d..245965c1b97a8 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 static String indexName(String tblName, QueryIndex idx) { /** * 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> mustDeserializeClss) throws IgniteCheckedException { GridKernalContext ctx = cctx.kernalContext(); CacheConfiguration ccfg = cctx.config(); @@ -179,7 +179,7 @@ public static QueryTypeCandidate typeForQueryEntity(String space, GridCacheConte 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 static QueryTypeCandidate typeForQueryEntity(String space, GridCacheConte 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 static QueryTypeCandidate typeForQueryEntity(String space, GridCacheConte 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); 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 58c909d09f9f1..b3fa47c5e81b9 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 FilteringVisitorClosure(SchemaIndexCacheVisitorClosure target) { /** {@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); } } 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 726c90ca75044..c45e2291df43f 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 UUID id() { } /** - * @return Space. + * @return Cache name. */ public String cacheName() { return cacheName; 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 935feabba0484..74d349a459348 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 @@ */ 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 IgniteBiPredicate forSpace(String spaceName); + @Nullable public IgniteBiPredicate 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. */ 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 4d53bea73ac88..8ec4a67a77da4 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.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. *

* NOTE: 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> query(@Nullable String spaceName, Collection params, + public Iterator> query(@Nullable String cacheName, Collection 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; } 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 0ed7e3382501b..5677f553f0d1e 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 @@ @IgniteSpiNoop public class NoopIndexingSpi extends IgniteSpiAdapter implements IndexingSpi { /** {@inheritDoc} */ - @Override public Iterator> query(@Nullable String spaceName, Collection params, + @Override public Iterator> query(@Nullable String cacheName, Collection 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; } 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 1d2752496d430..ac294b05067fb 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 static void forceFail(boolean failFlag) { } /** {@inheritDoc} */ - @Override public Iterator> query(@Nullable String spaceName, Collection params, + @Override public Iterator> query(@Nullable String cacheName, Collection 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 static void forceFail(boolean failFlag) { } /** {@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; 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 7349a4e82a763..b6e32d57cee76 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 @@ private static class MyIndexingSpi extends IgniteSpiAdapter implements IndexingS } /** {@inheritDoc} */ - @Override public Iterator> query(@Nullable String spaceName, Collection params, + @Override public Iterator> query(@Nullable String cacheName, Collection params, @Nullable IndexingQueryFilter filters) throws IgniteSpiException { if (params.size() < 2) throw new IgniteSpiException("Range parameters required."); @@ -278,7 +278,7 @@ private static class MyIndexingSpi extends IgniteSpiAdapter implements IndexingS } /** {@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 @@ private static class MyIndexingSpi extends IgniteSpiAdapter implements IndexingS } /** {@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 @@ private static class MyIndexingSpi extends IgniteSpiAdapter implements IndexingS 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 @@ private static class MyBinaryIndexingSpi extends MyIndexingSpi { */ 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"); } 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 9d2b31cbf4742..e59deed2eb767 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 @@ private static class MyBrokenIndexingSpi extends IgniteSpiAdapter implements Ind } /** {@inheritDoc} */ - @Override public Iterator> query(@Nullable String spaceName, Collection params, + @Override public Iterator> query(@Nullable String cacheName, Collection 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. } } 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 2a3d77cf264bb..db7bfd672a953 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 void start(GridKernalContext ctx, IgniteH2Indexing idx) { /** * 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 @@ else if (items == 0L) } /** - * @param spaceName Space name. + * @param cacheName Cache name. * @param stmt Prepared statement. * @param fieldsQry Initial query. * @param cancel Query cancel. @@ -202,9 +202,9 @@ else if (items == 0L) * @throws IgniteCheckedException if failed. */ @SuppressWarnings("unchecked") - QueryCursorImpl> updateSqlFieldsTwoStep(String spaceName, PreparedStatement stmt, + QueryCursorImpl> 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> resCur = (QueryCursorImpl>)new QueryCursorImpl(Collections.singletonList (Collections.singletonList(res.cnt)), null, false); @@ -216,17 +216,17 @@ QueryCursorImpl> updateSqlFieldsTwoStep(String spaceName, PreparedStatem /** * 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 @@ long streamUpdateQuery(IgniteDataStreamer streamer, PreparedStatement stmt, Obje * 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 @@ private UpdateResult executeUpdateStatement(final GridCacheContext cctx, Prepare /** * 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 spacePlans = planCache.get(spaceName); + ConcurrentMap 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 @@ private UpdatePlan getPlanForStatement(String spaceName, PreparedStatement prepS // 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; } 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 c099d77b33329..0ce905bf09694 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 @@ -228,10 +228,7 @@ /** * Indexing implementation based on H2 database engine. In this implementation main query language is SQL, - * fulltext indexing can be performed using Lucene. For each registered space - * the SPI will create respective schema, for default space (where space name is null) schema - * with name {@code ""} will be used. To avoid name conflicts user should not explicitly name - * a schema {@code ""}. + * fulltext indexing can be performed using Lucene. *

* For each registered {@link GridQueryTypeDescriptor} this SPI will create respective SQL table with * {@code '_key'} and {@code '_val'} fields for key and value, and fields from @@ -336,8 +333,8 @@ public class IgniteH2Indexing implements GridQueryIndexing { /** */ private GridReduceQueryExecutor rdcQryExec; - /** space name -> schema name */ - private final Map space2schema = new ConcurrentHashMap8<>(); + /** Cache name -> schema name */ + private final Map cacheName2schema = new ConcurrentHashMap8<>(); /** */ private AtomicLong qryIdGen; @@ -429,12 +426,12 @@ public GridKernalContext kernalContext() { } /** - * @param space Space. + * @param cacheName Cache name. * @return Connection. */ - public Connection connectionForSpace(String space) { + public Connection connectionForCache(String cacheName) { try { - return connectionForThread(schema(space)); + return connectionForThread(schema(cacheName)); } catch (IgniteCheckedException e) { throw new IgniteException(e); @@ -484,13 +481,13 @@ private PreparedStatement prepareStatement(Connection c, String sql, boolean use } /** {@inheritDoc} */ - @Override public PreparedStatement prepareNativeStatement(String space, String sql) throws SQLException { - return prepareStatement(connectionForSpace(space), sql, true); + @Override public PreparedStatement prepareNativeStatement(String cacheName, String sql) throws SQLException { + return prepareStatement(connectionForCache(cacheName), sql, true); } /** {@inheritDoc} */ @SuppressWarnings("unchecked") - @Override public IgniteDataStreamer createStreamer(String spaceName, PreparedStatement nativeStmt, + @Override public IgniteDataStreamer createStreamer(String cacheName, PreparedStatement nativeStmt, long autoFlushFreq, int nodeBufSize, int nodeParOps, boolean allowOverwrite) { Prepared prep = GridSqlQueryParser.prepared(nativeStmt); @@ -498,7 +495,7 @@ private PreparedStatement prepareStatement(Connection c, String sql, boolean use throw new IgniteSQLException("Only INSERT operations are supported in streaming mode", IgniteQueryErrorCode.UNSUPPORTED_OPERATION); - IgniteDataStreamer streamer = ctx.grid().dataStreamer(spaceName); + IgniteDataStreamer streamer = ctx.grid().dataStreamer(cacheName); streamer.autoFlushFrequency(autoFlushFreq); @@ -640,7 +637,7 @@ private void onSqlException() { } /** {@inheritDoc} */ - @Override public void store(String spaceName, + @Override public void store(String cacheName, String typeName, KeyCacheObject k, int partId, @@ -648,7 +645,7 @@ private void onSqlException() { GridCacheVersion ver, long expirationTime, long link) throws IgniteCheckedException { - TableDescriptor tbl = tableDescriptor(typeName, spaceName); + TableDescriptor tbl = tableDescriptor(typeName, cacheName); if (tbl == null) return; // Type was rejected. @@ -674,29 +671,29 @@ private boolean isBinary(CacheObject o) { } /** - * @param space Space. + * @param cacheName Cache name. * @return Cache object context. */ - private CacheObjectContext objectContext(String space) { + private CacheObjectContext objectContext(String cacheName) { if (ctx == null) return null; - return ctx.cache().internalCache(space).context().cacheObjectContext(); + return ctx.cache().internalCache(cacheName).context().cacheObjectContext(); } /** - * @param space Space. + * @param cacheName Cache name. * @return Cache object context. */ - private GridCacheContext cacheContext(String space) { + private GridCacheContext cacheContext(String cacheName) { if (ctx == null) return null; - return ctx.cache().internalCache(space).context(); + return ctx.cache().internalCache(cacheName).context(); } /** {@inheritDoc} */ - @Override public void remove(String spaceName, + @Override public void remove(String cacheName, GridQueryTypeDescriptor type, KeyCacheObject key, int partId, @@ -705,7 +702,7 @@ private GridCacheContext cacheContext(String space) { if (log.isDebugEnabled()) log.debug("Removing key from cache query index [locId=" + nodeId + ", key=" + key + ", val=" + val + ']'); - TableDescriptor tbl = tableDescriptor(type.name(), spaceName); + TableDescriptor tbl = tableDescriptor(type.name(), cacheName); if (tbl == null) return; @@ -764,12 +761,12 @@ private void removeTable(TableDescriptor tbl) throws IgniteCheckedException { /** * Add initial user index. * - * @param spaceName Space name. + * @param cacheName Cache name. * @param desc Table descriptor. * @param h2Idx User index. * @throws IgniteCheckedException If failed. */ - private void addInitialUserIndex(String spaceName, TableDescriptor desc, GridH2IndexBase h2Idx) + private void addInitialUserIndex(String cacheName, TableDescriptor desc, GridH2IndexBase h2Idx) throws IgniteCheckedException { GridH2Table h2Tbl = desc.tbl; @@ -778,7 +775,7 @@ private void addInitialUserIndex(String spaceName, TableDescriptor desc, GridH2I try { String sql = indexCreateSql(desc.fullTableName(), h2Idx, false, desc.schema.escapeAll()); - executeSql(spaceName, sql); + executeSql(cacheName, sql); } catch (Exception e) { // Rollback and re-throw. @@ -789,11 +786,11 @@ private void addInitialUserIndex(String spaceName, TableDescriptor desc, GridH2I } /** {@inheritDoc} */ - @Override public void dynamicIndexCreate(final String spaceName, final String tblName, + @Override public void dynamicIndexCreate(final String cacheName, final String tblName, final QueryIndexDescriptorImpl idxDesc, boolean ifNotExists, SchemaIndexCacheVisitor cacheVisitor) throws IgniteCheckedException { // Locate table. - String schemaName = schema(spaceName); + String schemaName = schema(cacheName); Schema schema = schemas.get(schemaName); @@ -834,7 +831,7 @@ private void addInitialUserIndex(String spaceName, TableDescriptor desc, GridH2I // prepared statements are re-built. String sql = indexCreateSql(desc.fullTableName(), h2Idx, ifNotExists, schema.escapeAll()); - executeSql(spaceName, sql); + executeSql(cacheName, sql); } catch (Exception e) { // Rollback and re-throw. @@ -846,27 +843,27 @@ private void addInitialUserIndex(String spaceName, TableDescriptor desc, GridH2I /** {@inheritDoc} */ @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter") - @Override public void dynamicIndexDrop(final String spaceName, String idxName, boolean ifExists) + @Override public void dynamicIndexDrop(final String cacheName, String idxName, boolean ifExists) throws IgniteCheckedException{ - String schemaName = schema(spaceName); + String schemaName = schema(cacheName); Schema schema = schemas.get(schemaName); String sql = indexDropSql(schemaName, idxName, ifExists, schema.escapeAll()); - executeSql(spaceName, sql); + executeSql(cacheName, sql); } /** * Execute DDL command. * - * @param spaceName Space name. + * @param cacheName Cache name. * @param sql SQL. * @throws IgniteCheckedException If failed. */ - private void executeSql(String spaceName, String sql) throws IgniteCheckedException { + private void executeSql(String cacheName, String sql) throws IgniteCheckedException { try { - Connection conn = connectionForSpace(spaceName); + Connection conn = connectionForCache(cacheName); try (PreparedStatement stmt = prepareStatement(conn, sql, false)) { stmt.execute(); @@ -984,12 +981,12 @@ private GridH2IndexBase createSpatialIndex(GridH2Table tbl, String idxName, Inde @SuppressWarnings("unchecked") @Override public GridCloseableIterator> queryLocalText( - String spaceName, String qry, String typeName, + String cacheName, String qry, String typeName, IndexingQueryFilter filters) throws IgniteCheckedException { - TableDescriptor tbl = tableDescriptor(typeName, spaceName); + TableDescriptor tbl = tableDescriptor(typeName, cacheName); if (tbl != null && tbl.luceneIdx != null) { - GridRunningQueryInfo run = new GridRunningQueryInfo(qryIdGen.incrementAndGet(), qry, TEXT, spaceName, + GridRunningQueryInfo run = new GridRunningQueryInfo(qryIdGen.incrementAndGet(), qry, TEXT, cacheName, U.currentTimeMillis(), null, true); try { @@ -1006,9 +1003,9 @@ private GridH2IndexBase createSpatialIndex(GridH2Table tbl, String idxName, Inde } /** {@inheritDoc} */ - @Override public void unregisterType(String spaceName, String typeName) + @Override public void unregisterType(String cacheName, String typeName) throws IgniteCheckedException { - TableDescriptor tbl = tableDescriptor(typeName, spaceName); + TableDescriptor tbl = tableDescriptor(typeName, cacheName); if (tbl != null) removeTable(tbl); @@ -1017,10 +1014,10 @@ private GridH2IndexBase createSpatialIndex(GridH2Table tbl, String idxName, Inde /** * Queries individual fields (generally used by JDBC drivers). * - * @param spaceName Space name. + * @param cacheName Cache name. * @param qry Query. * @param params Query parameters. - * @param filter Space name and key filter. + * @param filter Cache name and key filter. * @param enforceJoinOrder Enforce join order of tables in the query. * @param timeout Query timeout in milliseconds. * @param cancel Query cancel. @@ -1028,11 +1025,11 @@ private GridH2IndexBase createSpatialIndex(GridH2Table tbl, String idxName, Inde * @throws IgniteCheckedException If failed. */ @SuppressWarnings("unchecked") - public GridQueryFieldsResult queryLocalSqlFields(final String spaceName, final String qry, + public GridQueryFieldsResult queryLocalSqlFields(final String cacheName, final String qry, @Nullable final Collection params, final IndexingQueryFilter filter, boolean enforceJoinOrder, final int timeout, final GridQueryCancel cancel) throws IgniteCheckedException { - final Connection conn = connectionForSpace(spaceName); + final Connection conn = connectionForCache(cacheName); setupConnection(conn, false, enforceJoinOrder); @@ -1049,7 +1046,7 @@ public GridQueryFieldsResult queryLocalSqlFields(final String spaceName, final S fldsQry.setEnforceJoinOrder(enforceJoinOrder); fldsQry.setTimeout(timeout, TimeUnit.MILLISECONDS); - return dmlProc.updateLocalSqlFields(spaceName, stmt, fldsQry, filter, cancel); + return dmlProc.updateLocalSqlFields(cacheName, stmt, fldsQry, filter, cancel); } else if (DdlStatementsProcessor.isDdlStatement(p)) throw new IgniteSQLException("DDL statements are supported for the whole cluster only", @@ -1074,12 +1071,12 @@ else if (DdlStatementsProcessor.isDdlStatement(p)) GridH2QueryContext.set(ctx); GridRunningQueryInfo run = new GridRunningQueryInfo(qryIdGen.incrementAndGet(), qry, SQL_FIELDS, - spaceName, U.currentTimeMillis(), cancel, true); + cacheName, U.currentTimeMillis(), cancel, true); runs.putIfAbsent(run.id(), run); try { - ResultSet rs = executeSqlQueryWithTimer(spaceName, stmt, conn, qry, params, timeout, cancel); + ResultSet rs = executeSqlQueryWithTimer(cacheName, stmt, conn, qry, params, timeout, cancel); return new FieldsIterator(rs); } @@ -1093,9 +1090,9 @@ else if (DdlStatementsProcessor.isDdlStatement(p)) } /** {@inheritDoc} */ - @Override public long streamUpdateQuery(String spaceName, String qry, + @Override public long streamUpdateQuery(String cacheName, String qry, @Nullable Object[] params, IgniteDataStreamer streamer) throws IgniteCheckedException { - final Connection conn = connectionForSpace(spaceName); + final Connection conn = connectionForCache(cacheName); final PreparedStatement stmt; @@ -1235,7 +1232,7 @@ private ResultSet executeSqlQuery(final Connection conn, final PreparedStatement /** * Executes sql query and prints warning if query is too slow.. * - * @param space Space name. + * @param cacheName Cache name. * @param conn Connection,. * @param sql Sql query. * @param params Parameters. @@ -1244,21 +1241,21 @@ private ResultSet executeSqlQuery(final Connection conn, final PreparedStatement * @return Result. * @throws IgniteCheckedException If failed. */ - public ResultSet executeSqlQueryWithTimer(String space, + public ResultSet executeSqlQueryWithTimer(String cacheName, Connection conn, String sql, @Nullable Collection params, boolean useStmtCache, int timeoutMillis, @Nullable GridQueryCancel cancel) throws IgniteCheckedException { - return executeSqlQueryWithTimer(space, preparedStatementWithParams(conn, sql, params, useStmtCache), + return executeSqlQueryWithTimer(cacheName, preparedStatementWithParams(conn, sql, params, useStmtCache), conn, sql, params, timeoutMillis, cancel); } /** * Executes sql query and prints warning if query is too slow. * - * @param space Space name. + * @param cacheName Cache name. * @param stmt Prepared statement for query. * @param conn Connection. * @param sql Sql query. @@ -1267,7 +1264,7 @@ public ResultSet executeSqlQueryWithTimer(String space, * @return Result. * @throws IgniteCheckedException If failed. */ - private ResultSet executeSqlQueryWithTimer(String space, PreparedStatement stmt, + private ResultSet executeSqlQueryWithTimer(String cacheName, PreparedStatement stmt, Connection conn, String sql, @Nullable Collection params, @@ -1280,7 +1277,7 @@ private ResultSet executeSqlQueryWithTimer(String space, PreparedStatement stmt, long time = U.currentTimeMillis() - start; - long longQryExecTimeout = schemas.get(schema(space)).ccfg.getLongQueryWarningTimeout(); + long longQryExecTimeout = schemas.get(schema(cacheName)).ccfg.getLongQueryWarningTimeout(); if (time > longQryExecTimeout) { String msg = "Query execution is too long (" + time + " ms): " + sql; @@ -1350,11 +1347,11 @@ public static void setupConnection(Connection conn, boolean distributedJoins, bo else { final boolean keepBinary = cctx.keepBinary(); - final String space = cctx.name(); + final String cacheName = cctx.name(); final String sql = qry.getSql(); final Object[] args = qry.getArgs(); - final GridQueryFieldsResult res = queryLocalSqlFields(space, sql, F.asList(args), filter, + final GridQueryFieldsResult res = queryLocalSqlFields(cacheName, sql, F.asList(args), filter, qry.isEnforceJoinOrder(), qry.getTimeout(), cancel); QueryCursorImpl> cursor = new QueryCursorImpl<>(new Iterable>() { @@ -1385,7 +1382,7 @@ public static void setupConnection(Connection conn, boolean distributedJoins, bo return queryDistributedSql(cctx, qry); } else { - String space = cctx.name(); + String cacheName = cctx.name(); String type = qry.getType(); String sqlQry = qry.getSql(); String alias = qry.getAlias(); @@ -1393,7 +1390,7 @@ public static void setupConnection(Connection conn, boolean distributedJoins, bo GridQueryCancel cancel = new GridQueryCancel(); - final GridCloseableIterator> i = queryLocalSql(space, sqlQry, alias, + final GridCloseableIterator> i = queryLocalSql(cacheName, sqlQry, alias, F.asList(params), type, filter, cancel); return new QueryCursorImpl>(new Iterable>() { @@ -1427,19 +1424,19 @@ public static void setupConnection(Connection conn, boolean distributedJoins, bo /** * Executes regular query. * - * @param spaceName Space name. + * @param cacheName Cache name. * @param qry Query. * @param alias Table alias. * @param params Query parameters. * @param type Query return type. - * @param filter Space name and key filter. + * @param filter Cache name and key filter. * @return Queried rows. * @throws IgniteCheckedException If failed. */ - public GridCloseableIterator> queryLocalSql(String spaceName, + public GridCloseableIterator> queryLocalSql(String cacheName, final String qry, String alias, @Nullable final Collection params, String type, final IndexingQueryFilter filter, GridQueryCancel cancel) throws IgniteCheckedException { - final TableDescriptor tbl = tableDescriptor(type, spaceName); + final TableDescriptor tbl = tableDescriptor(type, cacheName); if (tbl == null) throw new IgniteSQLException("Failed to find SQL table for type: " + type, @@ -1454,13 +1451,13 @@ public GridCloseableIterator> queryLocalSql(String sp GridH2QueryContext.set(new GridH2QueryContext(nodeId, nodeId, 0, LOCAL).filter(filter) .distributedJoinMode(OFF)); - GridRunningQueryInfo run = new GridRunningQueryInfo(qryIdGen.incrementAndGet(), qry, SQL, spaceName, + GridRunningQueryInfo run = new GridRunningQueryInfo(qryIdGen.incrementAndGet(), qry, SQL, cacheName, U.currentTimeMillis(), null, true); runs.put(run.id(), run); try { - ResultSet rs = executeSqlQueryWithTimer(spaceName, conn, sql, params, true, 0, cancel); + ResultSet rs = executeSqlQueryWithTimer(cacheName, conn, sql, params, true, 0, cancel); return new KeyValIterator(rs); } @@ -1500,9 +1497,9 @@ private Iterable> runQueryTwoStep( @SuppressWarnings("unchecked") @Override public QueryCursor> queryDistributedSql(GridCacheContext cctx, SqlQuery qry) { String type = qry.getType(); - String space = cctx.name(); + String cacheName = cctx.name(); - TableDescriptor tblDesc = tableDescriptor(type, space); + TableDescriptor tblDesc = tableDescriptor(type, cacheName); if (tblDesc == null) throw new IgniteSQLException("Failed to find SQL table for type: " + type, @@ -1571,10 +1568,10 @@ public static Session session(Connection c) { /** {@inheritDoc} */ @Override public FieldsQueryCursor> queryDistributedSqlFields(GridCacheContext cctx, SqlFieldsQuery qry, GridQueryCancel cancel) { - final String space = cctx.name(); + final String cacheName = cctx.name(); final String sqlQry = qry.getSql(); - Connection c = connectionForSpace(space); + Connection c = connectionForCache(cacheName); final boolean enforceJoinOrder = qry.isEnforceJoinOrder(); final boolean distributedJoins = qry.isDistributedJoins(); @@ -1585,7 +1582,7 @@ public static Session session(Connection c) { GridCacheTwoStepQuery twoStepQry = null; List meta; - final TwoStepCachedQueryKey cachedQryKey = new TwoStepCachedQueryKey(space, sqlQry, grpByCollocated, + final TwoStepCachedQueryKey cachedQryKey = new TwoStepCachedQueryKey(cacheName, sqlQry, grpByCollocated, distributedJoins, enforceJoinOrder, qry.isLocal()); TwoStepCachedQuery cachedQry = twoStepCache.get(cachedQryKey); @@ -1680,7 +1677,7 @@ public static Session session(Connection c) { LinkedHashSet caches0 = new LinkedHashSet<>(); - // Setup spaces from schemas. + // Setup caches from schemas. assert twoStepQry != null; int tblCnt = twoStepQry.tablesCount(); @@ -1689,9 +1686,9 @@ public static Session session(Connection c) { caches0.add(cctx.cacheId()); for (QueryTable table : twoStepQry.tables()) { - String cacheName = cacheNameForSchemaAndTable(table.schema(), table.table()); + String tblCacheName = cacheNameForSchemaAndTable(table.schema(), table.table()); - int cacheId = CU.cacheId(cacheName); + int cacheId = CU.cacheId(tblCacheName); caches0.add(cacheId); } @@ -1752,8 +1749,7 @@ public static Session session(Connection c) { * @return Cache name. */ private String cacheNameForSchemaAndTable(String schemaName, String tblName) { - // TODO: This need to be changed. - return space(schemaName); + return cacheName(schemaName); } /** @@ -1848,11 +1844,11 @@ else if (star > 0) { * @param type Type description. * @throws IgniteCheckedException In case of error. */ - @Override public boolean registerType(String spaceName, GridQueryTypeDescriptor type) + @Override public boolean registerType(String cacheName, GridQueryTypeDescriptor type) throws IgniteCheckedException { validateTypeDescriptor(type); - String schemaName = schema(spaceName); + String schemaName = schema(cacheName); Schema schema = schemas.get(schemaName); @@ -1861,7 +1857,7 @@ else if (star > 0) { try { Connection conn = connectionForThread(schemaName); - createTable(spaceName, schema, tbl, conn); + createTable(cacheName, schema, tbl, conn); schema.add(tbl); } @@ -1969,14 +1965,14 @@ public static String escapeName(String name, boolean escapeAll) { /** * Create db table by using given table descriptor. * - * @param spaceName Space name. + * @param cacheName Cache name. * @param schema Schema. * @param tbl Table descriptor. * @param conn Connection. * @throws SQLException If failed to create db table. * @throws IgniteCheckedException If failed. */ - private void createTable(String spaceName, Schema schema, TableDescriptor tbl, Connection conn) + private void createTable(String cacheName, Schema schema, TableDescriptor tbl, Connection conn) throws SQLException, IgniteCheckedException { assert schema != null; assert tbl != null; @@ -2011,7 +2007,7 @@ private void createTable(String spaceName, Schema schema, TableDescriptor tbl, C GridH2Table h2Tbl = H2TableEngine.createTable(conn, sql.toString(), rowDesc, rowFactory, tbl); for (GridH2IndexBase usrIdx : tbl.createUserIndexes()) - addInitialUserIndex(spaceName, tbl, usrIdx); + addInitialUserIndex(cacheName, tbl, usrIdx); if (dataTables.putIfAbsent(h2Tbl.identifier(), h2Tbl) != null) throw new IllegalStateException("Table already exists: " + h2Tbl.identifierString()); @@ -2069,14 +2065,14 @@ private String dbTypeFromClass(Class cls) { } /** - * Gets table descriptor by type and space names. + * Gets table descriptor by type and cache names. * * @param type Type name. - * @param space Space name. + * @param cacheName Cache name. * @return Table descriptor. */ - @Nullable private TableDescriptor tableDescriptor(String type, String space) { - Schema s = schemas.get(schema(space)); + @Nullable private TableDescriptor tableDescriptor(String type, String cacheName) { + Schema s = schemas.get(schema(cacheName)); if (s == null) return null; @@ -2100,13 +2096,13 @@ private Collection tables(String schema) { } /** - * Gets database schema from space. + * Gets database schema from cache name. * - * @param space Space name. {@code null} would be converted to an empty string. - * @return Schema name. Should not be null since we should not fail for an invalid space name. + * @param cacheName Cache name. {@code null} would be converted to an empty string. + * @return Schema name. Should not be null since we should not fail for an invalid cache name. */ - private String schema(String space) { - return emptyIfNull(space2schema.get(emptyIfNull(space))); + private String schema(String cacheName) { + return emptyIfNull(cacheName2schema.get(emptyIfNull(cacheName))); } /** @@ -2127,7 +2123,7 @@ private void cleanupStatementCache() { } /** {@inheritDoc} */ - @Override public String space(String schemaName) { + @Override public String cacheName(String schemaName) { assert schemaName != null; Schema schema = schemas.get(schemaName); @@ -2139,19 +2135,19 @@ private void cleanupStatementCache() { schema = schemas.get(escapeName(schemaName, true)); } - return schema.spaceName; + return schema.cacheName; } /** * Rebuild indexes from hash index. * - * @param spaceName Space name. + * @param cacheName Cache name. * @param type Type descriptor. * @throws IgniteCheckedException If failed. */ - @Override public void rebuildIndexesFromHash(String spaceName, + @Override public void rebuildIndexesFromHash(String cacheName, GridQueryTypeDescriptor type) throws IgniteCheckedException { - TableDescriptor tbl = tableDescriptor(type.name(), spaceName); + TableDescriptor tbl = tableDescriptor(type.name(), cacheName); if (tbl == null) return; @@ -2208,8 +2204,8 @@ private void cleanupStatementCache() { } /** {@inheritDoc} */ - @Override public void markForRebuildFromHash(String spaceName, GridQueryTypeDescriptor type) { - TableDescriptor tbl = tableDescriptor(type.name(), spaceName); + @Override public void markForRebuildFromHash(String cacheName, GridQueryTypeDescriptor type) { + TableDescriptor tbl = tableDescriptor(type.name(), cacheName); if (tbl == null) return; @@ -2222,18 +2218,18 @@ private void cleanupStatementCache() { /** * Gets size (for tests only). * - * @param spaceName Space name. + * @param cacheName Cache name. * @param typeName Type name. * @return Size. * @throws IgniteCheckedException If failed or {@code -1} if the type is unknown. */ - long size(String spaceName, String typeName) throws IgniteCheckedException { - TableDescriptor tbl = tableDescriptor(typeName, spaceName); + long size(String cacheName, String typeName) throws IgniteCheckedException { + TableDescriptor tbl = tableDescriptor(typeName, cacheName); if (tbl == null) return -1; - Connection conn = connectionForSpace(spaceName); + Connection conn = connectionForCache(cacheName); setupConnection(conn, false, false); @@ -2519,7 +2515,7 @@ private void createSqlFunctions(String schema, Class[] clss) throws IgniteChe conns.clear(); schemas.clear(); - space2schema.clear(); + cacheName2schema.clear(); try (Connection c = DriverManager.getConnection(dbUrl); Statement s = c.createStatement()) { @@ -2539,14 +2535,14 @@ private void createSqlFunctions(String schema, Class[] clss) throws IgniteChe } /** {@inheritDoc} */ - @Override public void registerCache(String spaceName, GridCacheContext cctx, CacheConfiguration ccfg) + @Override public void registerCache(String cacheName, GridCacheContext cctx, CacheConfiguration ccfg) throws IgniteCheckedException { String schema = schemaNameFromCacheConf(ccfg); - if (schemas.putIfAbsent(schema, new Schema(spaceName, schema, cctx, ccfg)) != null) - throw new IgniteCheckedException("Cache already registered: " + U.maskName(spaceName)); + if (schemas.putIfAbsent(schema, new Schema(cacheName, schema, cctx, ccfg)) != null) + throw new IgniteCheckedException("Cache already registered: " + U.maskName(cacheName)); - space2schema.put(emptyIfNull(spaceName), schema); + cacheName2schema.put(emptyIfNull(cacheName), schema); createSchema(schema); @@ -2554,14 +2550,14 @@ private void createSqlFunctions(String schema, Class[] clss) throws IgniteChe } /** {@inheritDoc} */ - @Override public void unregisterCache(String spaceName) { - String schema = schema(spaceName); + @Override public void unregisterCache(String cacheName) { + String schema = schema(cacheName); Schema rmv = schemas.remove(schema); if (rmv != null) { - space2schema.remove(emptyIfNull(rmv.spaceName)); - mapQryExec.onCacheStop(spaceName); - dmlProc.onCacheStop(spaceName); + cacheName2schema.remove(emptyIfNull(rmv.cacheName)); + mapQryExec.onCacheStop(cacheName); + dmlProc.onCacheStop(cacheName); rmv.onDrop(); @@ -2569,7 +2565,7 @@ private void createSqlFunctions(String schema, Class[] clss) throws IgniteChe dropSchema(schema); } catch (IgniteCheckedException e) { - U.error(log, "Failed to drop schema on cache stop (will ignore): " + U.maskName(spaceName), e); + U.error(log, "Failed to drop schema on cache stop (will ignore): " + U.maskName(cacheName), e); } for (TableDescriptor tblDesc : rmv.tbls.values()) @@ -2580,7 +2576,7 @@ private void createSqlFunctions(String schema, Class[] clss) throws IgniteChe it.hasNext(); ) { Map.Entry e = it.next(); - if (F.eq(e.getKey().space, spaceName)) + if (F.eq(e.getKey().cacheName, cacheName)) it.remove(); } } @@ -2594,8 +2590,8 @@ private void createSqlFunctions(String schema, Class[] clss) throws IgniteChe final AffinityTopologyVersion topVer0 = topVer != null ? topVer : AffinityTopologyVersion.NONE; return new IndexingQueryFilter() { - @Nullable @Override public IgniteBiPredicate forSpace(String spaceName) { - final GridCacheAdapter cache = ctx.cache().internalCache(spaceName); + @Nullable @Override public IgniteBiPredicate forCache(String cacheName) { + final GridCacheAdapter cache = ctx.cache().internalCache(cacheName); if (cache.context().isReplicated()) return null; @@ -2677,7 +2673,7 @@ public void awaitForReadyTopologyVersion(AffinityTopologyVersion topVer) throws */ private static final class TwoStepCachedQueryKey { /** */ - private final String space; + private final String cacheName; /** */ private final String sql; @@ -2695,20 +2691,20 @@ private static final class TwoStepCachedQueryKey { private final boolean isLocal; /** - * @param space Space. + * @param cacheName Cache name. * @param sql Sql. * @param grpByCollocated Collocated GROUP BY. * @param distributedJoins Distributed joins enabled. * @param enforceJoinOrder Enforce join order of tables. * @param isLocal Query is local flag. */ - private TwoStepCachedQueryKey(String space, + private TwoStepCachedQueryKey(String cacheName, String sql, boolean grpByCollocated, boolean distributedJoins, boolean enforceJoinOrder, boolean isLocal) { - this.space = space; + this.cacheName = cacheName; this.sql = sql; this.grpByCollocated = grpByCollocated; this.distributedJoins = distributedJoins; @@ -2735,7 +2731,7 @@ private TwoStepCachedQueryKey(String space, if (enforceJoinOrder != that.enforceJoinOrder) return false; - if (space != null ? !space.equals(that.space) : that.space != null) + if (cacheName != null ? !cacheName.equals(that.cacheName) : that.cacheName != null) return false; return isLocal == that.isLocal && sql.equals(that.sql); @@ -2743,7 +2739,7 @@ private TwoStepCachedQueryKey(String space, /** {@inheritDoc} */ @Override public int hashCode() { - int res = space != null ? space.hashCode() : 0; + int res = cacheName != null ? cacheName.hashCode() : 0; res = 31 * res + sql.hashCode(); res = 31 * res + (grpByCollocated ? 1 : 0); res = res + (distributedJoins ? 2 : 0); @@ -3165,7 +3161,7 @@ H2RowFactory rowFactory(GridH2RowDescriptor rowDesc) { if (type().valueClass() == String.class) { try { - luceneIdx = new GridLuceneIndex(ctx, schema.offheap, schema.spaceName, type); + luceneIdx = new GridLuceneIndex(ctx, schema.offheap, schema.cacheName, type); } catch (IgniteCheckedException e1) { throw new IgniteException(e1); @@ -3178,7 +3174,7 @@ H2RowFactory rowFactory(GridH2RowDescriptor rowDesc) { if (textIdx != null) { try { - luceneIdx = new GridLuceneIndex(ctx, schema.offheap, schema.spaceName, type); + luceneIdx = new GridLuceneIndex(ctx, schema.offheap, schema.cacheName, type); } catch (IgniteCheckedException e1) { throw new IgniteException(e1); @@ -3450,7 +3446,7 @@ public SqlFieldMetadata() { */ private class Schema { /** */ - private final String spaceName; + private final String cacheName; /** */ private final String schemaName; @@ -3471,14 +3467,14 @@ private class Schema { private final CacheConfiguration ccfg; /** - * @param spaceName Space name. + * @param cacheName Cache name. * @param schemaName Schema name. * @param cctx Cache context. * @param ccfg Cache configuration. */ - private Schema(String spaceName, String schemaName, GridCacheContext cctx, + private Schema(String cacheName, String schemaName, GridCacheContext cctx, CacheConfiguration ccfg) { - this.spaceName = spaceName; + this.cacheName = cacheName; this.cctx = cctx; this.schemaName = schemaName; this.ccfg = ccfg; @@ -3653,9 +3649,9 @@ private class RowDescriptor implements GridH2RowDescriptor { CacheObject co = (CacheObject)obj; if (type == Value.JAVA_OBJECT) - return new GridH2ValueCacheObject(cacheContext(schema.spaceName), co); + return new GridH2ValueCacheObject(cacheContext(schema.cacheName), co); - obj = co.value(objectContext(schema.spaceName), false); + obj = co.value(objectContext(schema.cacheName), false); } switch (type) { @@ -3733,7 +3729,7 @@ private class RowDescriptor implements GridH2RowDescriptor { "or configure key type as common super class for all actual keys for this value type.", e); } - GridCacheContext cctx = cacheContext(schema.spaceName); + GridCacheContext cctx = cacheContext(schema.cacheName); if (cctx.offheapIndex()) { row.ver = ver; @@ -4013,7 +4009,7 @@ public static synchronized GridH2Table createTable(Connection conn, String sql, /** {@inheritDoc} */ @Override public TableBase createTable(CreateTableData createTblData) { - resTbl0 = new GridH2Table(createTblData, rowDesc0, rowFactory0, tblDesc0, tblDesc0.schema.spaceName); + resTbl0 = new GridH2Table(createTblData, rowDesc0, rowFactory0, tblDesc0, tblDesc0.schema.cacheName); return resTbl0; } diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2PkHashIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2PkHashIndex.java index 7579fa59c0e7c..7caf3541ba25f 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2PkHashIndex.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2PkHashIndex.java @@ -90,9 +90,9 @@ public H2PkHashIndex( IgniteBiPredicate p = null; if (f != null) { - String spaceName = getTable().cacheName(); + String cacheName = getTable().cacheName(); - p = f.forSpace(spaceName); + p = f.forCache(cacheName); } KeyCacheObject lowerObj = null; diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java index 814df7796a89b..c1c1d9c05d4ee 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java @@ -167,9 +167,9 @@ private List getAvailableInlineColumns(IndexColumn[] cols) { IgniteBiPredicate p = null; if (f != null) { - String spaceName = getTable().cacheName(); + String cacheName = getTable().cacheName(); - p = f.forSpace(spaceName); + p = f.forCache(cacheName); } int seg = threadLocalSegment(); diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java index 30fb612d4308e..3dabc58df7c03 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java @@ -1686,18 +1686,16 @@ protected static class FilteringCursor implements GridCursor { * @param cursor GridCursor. * @param time Time for expired rows filtering. * @param qryFilter Filter. - * @param spaceName Space name. + * @param cacheName Cache name. */ - protected FilteringCursor(GridCursor cursor, - long time, - IndexingQueryFilter qryFilter, - String spaceName) { + protected FilteringCursor(GridCursor cursor, long time, IndexingQueryFilter qryFilter, + String cacheName) { this.cursor = cursor; this.time = time; if (qryFilter != null) { - this.fltr = qryFilter.forSpace(spaceName); + this.fltr = qryFilter.forCache(cacheName); this.isValRequired = qryFilter.isValueRequired(); } diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java index ec728de291632..cb9042e841a44 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java @@ -222,7 +222,7 @@ public GridH2RowDescriptor rowDescriptor() { } /** - * @return Space name. + * @return Cache name. */ @Nullable public String cacheName() { return cacheName; diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java index 39256d457571c..03fedcbfad728 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java @@ -206,7 +206,7 @@ public GridH2TreeIndex(String name, GridH2Table tbl, boolean pk, List GridCloseableIterator> query(String qry, IgniteBiPredicate fltr = null; if (filters != null) - fltr = filters.forSpace(spaceName); + fltr = filters.forCache(cacheName); return new It<>(reader, searcher, docs.scoreDocs, fltr); } @@ -382,7 +381,7 @@ private void findNext() throws IgniteCheckedException { ClassLoader ldr = null; if (ctx != null && ctx.deploy().enabled()) - ldr = ctx.cache().internalCache(spaceName).context().deploy().globalLoader(); + ldr = ctx.cache().internalCache(cacheName).context().deploy().globalLoader(); K k = unmarshall(org.apache.commons.codec.binary.Base64.decodeBase64(doc.get(KEY_FIELD_NAME)), ldr); diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java index 6d76eeaa786ec..1d8a76293de9a 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java @@ -579,7 +579,7 @@ private void onQueryRequest0( } } - Connection conn = h2.connectionForSpace(mainCctx.name()); + Connection conn = h2.connectionForCache(mainCctx.name()); setupConnection(conn, distributedJoinMode != OFF, enforceJoinOrder); diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java index 75914ef797de4..1ecbb402c7fbc 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java @@ -540,10 +540,10 @@ public Iterator> query( final long qryReqId = qryIdGen.incrementAndGet(); - final String space = cctx.name(); + final String cacheName = cctx.name(); - final QueryRun r = new QueryRun(qryReqId, qry.originalSql(), space, - h2.connectionForSpace(space), qry.mapQueries().size(), qry.pageSize(), + final QueryRun r = new QueryRun(qryReqId, qry.originalSql(), cacheName, + h2.connectionForCache(cacheName), qry.mapQueries().size(), qry.pageSize(), U.currentTimeMillis(), cancel); AffinityTopologyVersion topVer = h2.readyTopologyVersion(); @@ -794,11 +794,11 @@ public Iterator> query( try { if (qry.explain()) - return explainPlan(r.conn, space, qry, params); + return explainPlan(r.conn, cacheName, qry, params); GridCacheSqlQuery rdc = qry.reduceQuery(); - ResultSet res = h2.executeSqlQueryWithTimer(space, + ResultSet res = h2.executeSqlQueryWithTimer(cacheName, r.conn, rdc.query(), F.asList(rdc.parameters(params)), @@ -1030,12 +1030,12 @@ private Collection replicatedUnstableDataNodes(List cacheI } /** - * @param space Cache name. + * @param cacheName Cache name. * @param topVer Topology version. * @return Collection of data nodes. */ - private Collection dataNodes(String space, AffinityTopologyVersion topVer) { - Collection res = ctx.discovery().cacheAffinityNodes(space, topVer); + private Collection dataNodes(String cacheName, AffinityTopologyVersion topVer) { + Collection res = ctx.discovery().cacheAffinityNodes(cacheName, topVer); return res != null ? res : Collections.emptySet(); } @@ -1049,12 +1049,12 @@ private Collection dataNodes(String space, AffinityTopologyVersion private Set replicatedUnstableDataNodes(GridCacheContext cctx) { assert cctx.isReplicated() : cctx.name() + " must be replicated"; - String space = cctx.name(); + String cacheName = cctx.name(); - Set dataNodes = new HashSet<>(dataNodes(space, NONE)); + Set dataNodes = new HashSet<>(dataNodes(cacheName, NONE)); if (dataNodes.isEmpty()) - throw new CacheException("Failed to find data nodes for cache: " + space); + throw new CacheException("Failed to find data nodes for cache: " + cacheName); // Find all the nodes owning all the partitions for replicated cache. for (int p = 0, parts = cctx.affinity().partitions(); p < parts; p++) { @@ -1213,18 +1213,18 @@ else if (!F.isEmpty(dataNodes(cctx.name(), NONE))) /** * @param c Connection. - * @param space Space. + * @param cacheName Cache name. * @param qry Query. * @param params Query parameters. * @return Cursor for plans. * @throws IgniteCheckedException if failed. */ - private Iterator> explainPlan(JdbcConnection c, String space, GridCacheTwoStepQuery qry, Object[] params) + private Iterator> explainPlan(JdbcConnection c, String cacheName, GridCacheTwoStepQuery qry, Object[] params) throws IgniteCheckedException { List> lists = new ArrayList<>(); for (int i = 0, mapQrys = qry.mapQueries().size(); i < mapQrys; i++) { - ResultSet rs = h2.executeSqlQueryWithTimer(space, c, + ResultSet rs = h2.executeSqlQueryWithTimer(cacheName, c, "SELECT PLAN FROM " + mergeTableIdentifier(i), null, false, 0, null); lists.add(F.asList(getPlan(rs))); @@ -1240,7 +1240,7 @@ private Iterator> explainPlan(JdbcConnection c, String space, GridCacheT GridCacheSqlQuery rdc = qry.reduceQuery(); - ResultSet rs = h2.executeSqlQueryWithTimer(space, + ResultSet rs = h2.executeSqlQueryWithTimer(cacheName, c, "EXPLAIN " + rdc.query(), F.asList(rdc.parameters(params)), diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractBasicSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractBasicSelfTest.java index 1241d0cef8a37..7e3473f571dd8 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractBasicSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractBasicSelfTest.java @@ -1020,13 +1020,13 @@ protected static void assertSchemaException(RunnableX r, int expCode) { /** * Synchronously create index. * - * @param space Space. + * @param cacheName Cache name. * @param tblName Table name. * @param idx Index. * @param ifNotExists When set to true operation will fail if index already exists. * @throws Exception If failed. */ - private void dynamicIndexCreate(String space, String tblName, QueryIndex idx, boolean ifNotExists) + private void dynamicIndexCreate(String cacheName, String tblName, QueryIndex idx, boolean ifNotExists) throws Exception { GridStringBuilder sql = new SB("CREATE INDEX ") .a(ifNotExists ? "IF NOT EXISTS " : "") @@ -1051,32 +1051,32 @@ private void dynamicIndexCreate(String space, String tblName, QueryIndex idx, bo sql.a(')'); - executeSql(space, sql.toString()); + executeSql(cacheName, sql.toString()); } /** * Synchronously drop index. * - * @param space Space. + * @param cacheName Cache name. * @param idxName Index name. * @param ifExists When set to true operation fill fail if index doesn't exists. * @throws Exception if failed. */ - private void dynamicIndexDrop(String space, String idxName, boolean ifExists) throws Exception { + private void dynamicIndexDrop(String cacheName, String idxName, boolean ifExists) throws Exception { String sql = "DROP INDEX " + (ifExists ? "IF EXISTS " : "") + "\"" + idxName + "\""; - executeSql(space, sql); + executeSql(cacheName, sql); } /** * Execute SQL. * - * @param space Space. + * @param cacheName Cache name. * @param sql SQL. */ - private void executeSql(String space, String sql) { + private void executeSql(String cacheName, String sql) { log.info("Executing DDL: " + sql); - node().cache(space).query(new SqlFieldsQuery(sql)).getAll(); + node().cache(cacheName).query(new SqlFieldsQuery(sql)).getAll(); } } diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractConcurrentSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractConcurrentSelfTest.java index f9e2f7586324f..3600022d7d0fa 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractConcurrentSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractConcurrentSelfTest.java @@ -1033,20 +1033,20 @@ private static void awaitIndexing(UUID nodeId) { */ private static class BlockingIndexing extends IgniteH2Indexing { /** {@inheritDoc} */ - @Override public void dynamicIndexCreate(@NotNull String spaceName, String tblName, + @Override public void dynamicIndexCreate(@NotNull String cacheName, String tblName, QueryIndexDescriptorImpl idxDesc, boolean ifNotExists, SchemaIndexCacheVisitor cacheVisitor) throws IgniteCheckedException { awaitIndexing(ctx.localNodeId()); - super.dynamicIndexCreate(spaceName, tblName, idxDesc, ifNotExists, cacheVisitor); + super.dynamicIndexCreate(cacheName, tblName, idxDesc, ifNotExists, cacheVisitor); } /** {@inheritDoc} */ - @Override public void dynamicIndexDrop(@NotNull String spaceName, String idxName, boolean ifExists) + @Override public void dynamicIndexDrop(@NotNull String cacheName, String idxName, boolean ifExists) throws IgniteCheckedException{ awaitIndexing(ctx.localNodeId()); - super.dynamicIndexDrop(spaceName, idxName, ifExists); + super.dynamicIndexDrop(cacheName, idxName, ifExists); } } } diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteQueryDedicatedPoolTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteQueryDedicatedPoolTest.java index ddaea8a91b16c..13c0cb2c5e73d 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteQueryDedicatedPoolTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteQueryDedicatedPoolTest.java @@ -191,7 +191,7 @@ private static class TestIndexingSpi extends IgniteSpiAdapter implements Indexin } /** {@inheritDoc} */ - @Override public Iterator> query(@Nullable String spaceName, Collection params, + @Override public Iterator> query(@Nullable String cacheName, Collection params, @Nullable IndexingQueryFilter filters) { return idx.containsKey(GridIoPolicy.QUERY_POOL) ? Collections.>singletonList( @@ -200,12 +200,12 @@ private static class TestIndexingSpi extends IgniteSpiAdapter implements Indexin } /** {@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) { idx.put(key, val); } /** {@inheritDoc} */ - @Override public void remove(@Nullable String spaceName, Object key) { + @Override public void remove(@Nullable String cacheName, Object key) { // No-op. } } 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 3832878519a03..7ba7d567396a0 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 @@ -246,32 +246,32 @@ private KeyCacheObject key(int key) { public void testSpi() throws Exception { IgniteH2Indexing spi = getIndexing(); - assertEquals(-1, spi.size(typeAA.space(), typeAA.name())); - assertEquals(-1, spi.size(typeAB.space(), typeAB.name())); - assertEquals(-1, spi.size(typeBA.space(), typeBA.name())); + assertEquals(-1, spi.size(typeAA.cacheName(), typeAA.name())); + assertEquals(-1, spi.size(typeAB.cacheName(), typeAB.name())); + assertEquals(-1, spi.size(typeBA.cacheName(), typeBA.name())); IgniteCache cacheA = ignite0.createCache(cacheACfg()); - assertEquals(0, spi.size(typeAA.space(), typeAA.name())); - assertEquals(0, spi.size(typeAB.space(), typeAB.name())); - assertEquals(-1, spi.size(typeBA.space(), typeBA.name())); + assertEquals(0, spi.size(typeAA.cacheName(), typeAA.name())); + assertEquals(0, spi.size(typeAB.cacheName(), typeAB.name())); + assertEquals(-1, spi.size(typeBA.cacheName(), typeBA.name())); IgniteCache cacheB = ignite0.createCache(cacheBCfg()); // Initially all is empty. - assertEquals(0, spi.size(typeAA.space(), typeAA.name())); - assertEquals(0, spi.size(typeAB.space(), typeAB.name())); - assertEquals(0, spi.size(typeBA.space(), typeBA.name())); + assertEquals(0, spi.size(typeAA.cacheName(), typeAA.name())); + assertEquals(0, spi.size(typeAB.cacheName(), typeAB.name())); + assertEquals(0, spi.size(typeBA.cacheName(), typeBA.name())); - assertFalse(spi.queryLocalSql(typeAA.space(), "select * from A.A", null, Collections.emptySet(), typeAA.name(), null, null).hasNext()); - assertFalse(spi.queryLocalSql(typeAB.space(), "select * from A.B", null, Collections.emptySet(), typeAB.name(), null, null).hasNext()); - assertFalse(spi.queryLocalSql(typeBA.space(), "select * from B.A", null, Collections.emptySet(), typeBA.name(), null, null).hasNext()); + assertFalse(spi.queryLocalSql(typeAA.cacheName(), "select * from A.A", null, Collections.emptySet(), typeAA.name(), null, null).hasNext()); + assertFalse(spi.queryLocalSql(typeAB.cacheName(), "select * from A.B", null, Collections.emptySet(), typeAB.name(), null, null).hasNext()); + assertFalse(spi.queryLocalSql(typeBA.cacheName(), "select * from B.A", null, Collections.emptySet(), typeBA.name(), null, null).hasNext()); - assertFalse(spi.queryLocalSql(typeBA.space(), "select * from B.A, A.B, A.A", null, + assertFalse(spi.queryLocalSql(typeBA.cacheName(), "select * from B.A, A.B, A.A", null, Collections.emptySet(), typeBA.name(), null, null).hasNext()); try { - spi.queryLocalSql(typeBA.space(), "select aa.*, ab.*, ba.* from A.A aa, A.B ab, B.A ba", null, + spi.queryLocalSql(typeBA.cacheName(), "select aa.*, ab.*, ba.* from A.A aa, A.B ab, B.A ba", null, Collections.emptySet(), typeBA.name(), null, null).hasNext(); fail("Enumerations of aliases in select block must be prohibited"); @@ -280,60 +280,60 @@ public void testSpi() throws Exception { // all fine } - assertFalse(spi.queryLocalSql(typeAB.space(), "select ab.* from A.B ab", null, + assertFalse(spi.queryLocalSql(typeAB.cacheName(), "select ab.* from A.B ab", null, Collections.emptySet(), typeAB.name(), null, null).hasNext()); - assertFalse(spi.queryLocalSql(typeBA.space(), "select ba.* from B.A as ba", null, + assertFalse(spi.queryLocalSql(typeBA.cacheName(), "select ba.* from B.A as ba", null, Collections.emptySet(), typeBA.name(), null, null).hasNext()); cacheA.put(1, aa("A", 1, "Vasya", 10).build()); - assertEquals(1, spi.size(typeAA.space(), typeAA.name())); - assertEquals(0, spi.size(typeAB.space(), typeAB.name())); - assertEquals(0, spi.size(typeBA.space(), typeBA.name())); + assertEquals(1, spi.size(typeAA.cacheName(), typeAA.name())); + assertEquals(0, spi.size(typeAB.cacheName(), typeAB.name())); + assertEquals(0, spi.size(typeBA.cacheName(), typeBA.name())); cacheA.put(1, ab(1, "Vasya", 20, "Some text about Vasya goes here.").build()); - // In one space all keys must be unique. - assertEquals(0, spi.size(typeAA.space(), typeAA.name())); - assertEquals(1, spi.size(typeAB.space(), typeAB.name())); - assertEquals(0, spi.size(typeBA.space(), typeBA.name())); + // In one cache all keys must be unique. + assertEquals(0, spi.size(typeAA.cacheName(), typeAA.name())); + assertEquals(1, spi.size(typeAB.cacheName(), typeAB.name())); + assertEquals(0, spi.size(typeBA.cacheName(), typeBA.name())); cacheB.put(1, ba(2, "Petya", 25, true).build()); - // No replacement because of different space. - assertEquals(0, spi.size(typeAA.space(), typeAA.name())); - assertEquals(1, spi.size(typeAB.space(), typeAB.name())); - assertEquals(1, spi.size(typeBA.space(), typeBA.name())); + // No replacement because of different cache. + assertEquals(0, spi.size(typeAA.cacheName(), typeAA.name())); + assertEquals(1, spi.size(typeAB.cacheName(), typeAB.name())); + assertEquals(1, spi.size(typeBA.cacheName(), typeBA.name())); cacheB.put(1, ba(2, "Kolya", 25, true).build()); // Replacement in the same table. - assertEquals(0, spi.size(typeAA.space(), typeAA.name())); - assertEquals(1, spi.size(typeAB.space(), typeAB.name())); - assertEquals(1, spi.size(typeBA.space(), typeBA.name())); + assertEquals(0, spi.size(typeAA.cacheName(), typeAA.name())); + assertEquals(1, spi.size(typeAB.cacheName(), typeAB.name())); + assertEquals(1, spi.size(typeBA.cacheName(), typeBA.name())); cacheA.put(2, aa("A", 2, "Valera", 19).build()); - assertEquals(1, spi.size(typeAA.space(), typeAA.name())); - assertEquals(1, spi.size(typeAB.space(), typeAB.name())); - assertEquals(1, spi.size(typeBA.space(), typeBA.name())); + assertEquals(1, spi.size(typeAA.cacheName(), typeAA.name())); + assertEquals(1, spi.size(typeAB.cacheName(), typeAB.name())); + assertEquals(1, spi.size(typeBA.cacheName(), typeBA.name())); cacheA.put(3, aa("A", 3, "Borya", 18).build()); - assertEquals(2, spi.size(typeAA.space(), typeAA.name())); - assertEquals(1, spi.size(typeAB.space(), typeAB.name())); - assertEquals(1, spi.size(typeBA.space(), typeBA.name())); + assertEquals(2, spi.size(typeAA.cacheName(), typeAA.name())); + assertEquals(1, spi.size(typeAB.cacheName(), typeAB.name())); + assertEquals(1, spi.size(typeBA.cacheName(), typeBA.name())); cacheA.put(4, ab(4, "Vitalya", 20, "Very Good guy").build()); - assertEquals(2, spi.size(typeAA.space(), typeAA.name())); - assertEquals(2, spi.size(typeAB.space(), typeAB.name())); - assertEquals(1, spi.size(typeBA.space(), typeBA.name())); + assertEquals(2, spi.size(typeAA.cacheName(), typeAA.name())); + assertEquals(2, spi.size(typeAB.cacheName(), typeAB.name())); + assertEquals(1, spi.size(typeBA.cacheName(), typeBA.name())); // Query data. Iterator> res = - spi.queryLocalSql(typeAA.space(), "from a order by age", null, Collections.emptySet(), typeAA.name(), null, null); + spi.queryLocalSql(typeAA.cacheName(), "from a order by age", null, Collections.emptySet(), typeAA.name(), null, null); assertTrue(res.hasNext()); assertEquals(aa("A", 3, "Borya", 18).build(), value(res.next())); @@ -341,7 +341,7 @@ public void testSpi() throws Exception { assertEquals(aa("A", 2, "Valera", 19).build(), value(res.next())); assertFalse(res.hasNext()); - res = spi.queryLocalSql(typeAA.space(), "select aa.* from a aa order by aa.age", null, + res = spi.queryLocalSql(typeAA.cacheName(), "select aa.* from a aa order by aa.age", null, Collections.emptySet(), typeAA.name(), null, null); assertTrue(res.hasNext()); @@ -350,7 +350,7 @@ public void testSpi() throws Exception { assertEquals(aa("A", 2, "Valera", 19).build(), value(res.next())); assertFalse(res.hasNext()); - res = spi.queryLocalSql(typeAB.space(), "from b order by name", null, Collections.emptySet(), typeAB.name(), null, null); + res = spi.queryLocalSql(typeAB.cacheName(), "from b order by name", null, Collections.emptySet(), typeAB.name(), null, null); assertTrue(res.hasNext()); assertEquals(ab(1, "Vasya", 20, "Some text about Vasya goes here.").build(), value(res.next())); @@ -358,7 +358,7 @@ public void testSpi() throws Exception { assertEquals(ab(4, "Vitalya", 20, "Very Good guy").build(), value(res.next())); assertFalse(res.hasNext()); - res = spi.queryLocalSql(typeAB.space(), "select bb.* from b as bb order by bb.name", null, + res = spi.queryLocalSql(typeAB.cacheName(), "select bb.* from b as bb order by bb.name", null, Collections.emptySet(), typeAB.name(), null, null); assertTrue(res.hasNext()); @@ -368,14 +368,14 @@ public void testSpi() throws Exception { assertFalse(res.hasNext()); - res = spi.queryLocalSql(typeBA.space(), "from a", null, Collections.emptySet(), typeBA.name(), null, null); + res = spi.queryLocalSql(typeBA.cacheName(), "from a", null, Collections.emptySet(), typeBA.name(), null, null); assertTrue(res.hasNext()); assertEquals(ba(2, "Kolya", 25, true).build(), value(res.next())); assertFalse(res.hasNext()); // Text queries - Iterator> txtRes = spi.queryLocalText(typeAB.space(), "good", + Iterator> txtRes = spi.queryLocalText(typeAB.cacheName(), "good", typeAB.name(), null); assertTrue(txtRes.hasNext()); @@ -410,32 +410,32 @@ public void testSpi() throws Exception { // Remove cacheA.remove(2); - assertEquals(1, spi.size(typeAA.space(), typeAA.name())); - assertEquals(2, spi.size(typeAB.space(), typeAB.name())); - assertEquals(1, spi.size(typeBA.space(), typeBA.name())); + assertEquals(1, spi.size(typeAA.cacheName(), typeAA.name())); + assertEquals(2, spi.size(typeAB.cacheName(), typeAB.name())); + assertEquals(1, spi.size(typeBA.cacheName(), typeBA.name())); cacheB.remove(1); - assertEquals(1, spi.size(typeAA.space(), typeAA.name())); - assertEquals(2, spi.size(typeAB.space(), typeAB.name())); - assertEquals(0, spi.size(typeBA.space(), typeBA.name())); + assertEquals(1, spi.size(typeAA.cacheName(), typeAA.name())); + assertEquals(2, spi.size(typeAB.cacheName(), typeAB.name())); + assertEquals(0, spi.size(typeBA.cacheName(), typeBA.name())); // Unregister. - spi.unregisterType(typeAA.space(), typeAA.name()); + spi.unregisterType(typeAA.cacheName(), typeAA.name()); - assertEquals(-1, spi.size(typeAA.space(), typeAA.name())); - assertEquals(2, spi.size(typeAB.space(), typeAB.name())); - assertEquals(0, spi.size(typeBA.space(), typeBA.name())); + assertEquals(-1, spi.size(typeAA.cacheName(), typeAA.name())); + assertEquals(2, spi.size(typeAB.cacheName(), typeAB.name())); + assertEquals(0, spi.size(typeBA.cacheName(), typeBA.name())); - spi.unregisterType(typeAB.space(), typeAB.name()); + spi.unregisterType(typeAB.cacheName(), typeAB.name()); - assertEquals(-1, spi.size(typeAA.space(), typeAA.name())); - assertEquals(-1, spi.size(typeAB.space(), typeAB.name())); - assertEquals(0, spi.size(typeBA.space(), typeBA.name())); + assertEquals(-1, spi.size(typeAA.cacheName(), typeAA.name())); + assertEquals(-1, spi.size(typeAB.cacheName(), typeAB.name())); + assertEquals(0, spi.size(typeBA.cacheName(), typeBA.name())); - spi.unregisterType(typeBA.space(), typeBA.name()); + spi.unregisterType(typeBA.cacheName(), typeBA.name()); - assertEquals(-1, spi.size(typeAA.space(), typeAA.name())); + assertEquals(-1, spi.size(typeAA.cacheName(), typeAA.name())); } /** @@ -533,7 +533,7 @@ private static class TypeDesc implements GridQueryTypeDescriptor { private final String name; /** */ - private final String space; + private final String cacheName; /** */ private final Map> valFields; @@ -542,14 +542,14 @@ private static class TypeDesc implements GridQueryTypeDescriptor { private final GridQueryIndexDescriptor textIdx; /** - * @param space Space name. + * @param cacheName Cache name. * @param name Type name. * @param valFields Fields. * @param textIdx Fulltext index. */ - private TypeDesc(String space, String name, Map> valFields, GridQueryIndexDescriptor textIdx) { + private TypeDesc(String cacheName, String name, Map> valFields, GridQueryIndexDescriptor textIdx) { this.name = name; - this.space = space; + this.cacheName = cacheName; this.valFields = Collections.unmodifiableMap(valFields); this.textIdx = textIdx; } @@ -570,10 +570,10 @@ private TypeDesc(String space, String name, Map> valFields, Gri } /** - * @return Space name. + * @return Cache name. */ - String space() { - return space; + String cacheName() { + return cacheName; } /** {@inheritDoc} */ diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java index 21666d619ac0c..98380842e40a4 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java @@ -686,7 +686,7 @@ private JdbcConnection connection() throws Exception { IgniteH2Indexing idx = U.field(qryProcessor, "idx"); - return (JdbcConnection)idx.connectionForSpace(DEFAULT_CACHE_NAME); + return (JdbcConnection)idx.connectionForCache(DEFAULT_CACHE_NAME); } /**