Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id BBB7F200C8D for ; Thu, 20 Apr 2017 10:39:15 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id BA72F160BB0; Thu, 20 Apr 2017 08:39:15 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 55102160BCC for ; Thu, 20 Apr 2017 10:39:13 +0200 (CEST) Received: (qmail 4939 invoked by uid 500); 20 Apr 2017 08:39:12 -0000 Mailing-List: contact commits-help@ignite.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@ignite.apache.org Delivered-To: mailing list commits@ignite.apache.org Received: (qmail 2144 invoked by uid 99); 20 Apr 2017 08:39:09 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 20 Apr 2017 08:39:09 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id C0EBFE10FF; Thu, 20 Apr 2017 08:39:09 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: agoncharuk@apache.org To: commits@ignite.apache.org Date: Thu, 20 Apr 2017 08:39:43 -0000 Message-Id: <74f3eadf19cb4dd19ddc43cdcea48b18@git.apache.org> In-Reply-To: <2e8d7ef222ea405998aa3e7df9518002@git.apache.org> References: <2e8d7ef222ea405998aa3e7df9518002@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [36/71] [abbrv] ignite git commit: IGNITE-4565: Implemented CREATE INDEX and DROP INDEX. This closes #1773. This closes #1804. archived-at: Thu, 20 Apr 2017 08:39:15 -0000 http://git-wip-us.apache.org/repos/asf/ignite/blob/2edb935c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java index 7841f06..da6ebc1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java @@ -60,7 +60,12 @@ import org.apache.ignite.configuration.NearCacheConfiguration; import org.apache.ignite.configuration.TransactionConfiguration; import org.apache.ignite.events.EventType; import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.processors.query.QuerySchema; import org.apache.ignite.internal.processors.query.QueryUtils; +import org.apache.ignite.internal.processors.query.schema.message.SchemaAbstractDiscoveryMessage; +import org.apache.ignite.internal.processors.query.schema.SchemaExchangeWorkerTask; +import org.apache.ignite.internal.processors.query.schema.SchemaNodeLeaveExchangeWorkerTask; +import org.apache.ignite.internal.processors.query.schema.message.SchemaProposeDiscoveryMessage; import org.apache.ignite.internal.suggestions.GridPerformanceSuggestions; import org.apache.ignite.internal.IgniteClientDisconnectedCheckedException; import org.apache.ignite.internal.IgniteComponentType; @@ -343,6 +348,13 @@ public class GridCacheProcessor extends GridProcessorAdapter { * @return Task or {@code null} if message doesn't require any special processing. */ public CachePartitionExchangeWorkerTask exchangeTaskForCustomDiscoveryMessage(DiscoveryCustomMessage msg) { + if (msg instanceof SchemaAbstractDiscoveryMessage) { + SchemaAbstractDiscoveryMessage msg0 = (SchemaAbstractDiscoveryMessage)msg; + + if (msg0.exchange()) + return new SchemaExchangeWorkerTask(msg0); + } + return null; } @@ -352,7 +364,24 @@ public class GridCacheProcessor extends GridProcessorAdapter { * @param task Task. */ public void processCustomExchangeTask(CachePartitionExchangeWorkerTask task) { - // No-op. + if (task instanceof SchemaExchangeWorkerTask) { + SchemaAbstractDiscoveryMessage msg = ((SchemaExchangeWorkerTask)task).message(); + + if (msg instanceof SchemaProposeDiscoveryMessage) { + SchemaProposeDiscoveryMessage msg0 = (SchemaProposeDiscoveryMessage)msg; + + ctx.query().onSchemaPropose(msg0); + } + else + U.warn(log, "Unsupported schema discovery message: " + msg); + } + else if (task instanceof SchemaNodeLeaveExchangeWorkerTask) { + SchemaNodeLeaveExchangeWorkerTask task0 = (SchemaNodeLeaveExchangeWorkerTask)task; + + ctx.query().onNodeLeave(task0.node()); + } + else + U.warn(log, "Unsupported custom exchange task: " + task); } /** @@ -719,7 +748,8 @@ public class GridCacheProcessor extends GridProcessorAdapter { cfg, cacheType, template, - IgniteUuid.randomUuid()); + IgniteUuid.randomUuid(), + new QuerySchema(cfg.getQueryEntities())); desc.locallyConfigured(true); desc.staticallyConfigured(true); @@ -755,7 +785,8 @@ public class GridCacheProcessor extends GridProcessorAdapter { cfg, cacheType, true, - IgniteUuid.randomUuid()); + IgniteUuid.randomUuid(), + new QuerySchema(cfg.getQueryEntities())); desc0.locallyConfigured(true); desc0.staticallyConfigured(true); @@ -831,6 +862,8 @@ public class GridCacheProcessor extends GridProcessorAdapter { // Must start database before start first cache. sharedCtx.database().onKernalStart(false); + ctx.query().onCacheKernalStart(); + // Start dynamic caches received from collect discovery data. for (DynamicCacheDescriptor desc : cacheDescriptors()) { if (ctx.config().isDaemon()) @@ -866,7 +899,7 @@ public class GridCacheProcessor extends GridProcessorAdapter { caches.put(maskNull(name), cache); - startCache(cache); + startCache(cache, desc.schema()); jCacheProxies.put(maskNull(name), new IgniteCacheProxy(ctx, cache, null, false)); } @@ -1177,10 +1210,11 @@ public class GridCacheProcessor extends GridProcessorAdapter { /** * @param cache Cache to start. + * @param schema Cache schema. * @throws IgniteCheckedException If failed to start cache. */ @SuppressWarnings({"TypeMayBeWeakened", "unchecked"}) - private void startCache(GridCacheAdapter cache) throws IgniteCheckedException { + private void startCache(GridCacheAdapter cache, QuerySchema schema) throws IgniteCheckedException { GridCacheContext cacheCtx = cache.context(); ctx.continuous().onCacheStart(cacheCtx); @@ -1220,7 +1254,7 @@ public class GridCacheProcessor extends GridProcessorAdapter { cacheCtx.cache().start(); - ctx.query().onCacheStart(cacheCtx); + ctx.query().onCacheStart(cacheCtx, schema); cacheCtx.onStarted(); @@ -1779,6 +1813,11 @@ public class GridCacheProcessor extends GridProcessorAdapter { assert req.start() : req; assert req.cacheType() != null : req; + DynamicCacheDescriptor desc = cacheDescriptor(req.cacheName()); + + if (desc != null) + desc.onStart(); + prepareCacheStart( req.startCacheConfiguration(), req.nearCacheConfiguration(), @@ -1786,13 +1825,9 @@ public class GridCacheProcessor extends GridProcessorAdapter { req.clientStartOnly(), req.initiatingNodeId(), req.deploymentId(), - topVer + topVer, + desc != null ? desc.schema() : null ); - - DynamicCacheDescriptor desc = cacheDescriptor(req.cacheName()); - - if (desc != null) - desc.onStart(); } /** @@ -1828,7 +1863,8 @@ public class GridCacheProcessor extends GridProcessorAdapter { false, null, desc.deploymentId(), - topVer + topVer, + desc.schema() ); } } @@ -1845,6 +1881,7 @@ public class GridCacheProcessor extends GridProcessorAdapter { * @param initiatingNodeId Initiating node ID. * @param deploymentId Deployment ID. * @param topVer Topology version. + * @param schema Query schema. * @throws IgniteCheckedException If failed. */ private void prepareCacheStart( @@ -1854,7 +1891,8 @@ public class GridCacheProcessor extends GridProcessorAdapter { boolean clientStartOnly, UUID initiatingNodeId, IgniteUuid deploymentId, - AffinityTopologyVersion topVer + AffinityTopologyVersion topVer, + @Nullable QuerySchema schema ) throws IgniteCheckedException { CacheConfiguration ccfg = new CacheConfiguration(cfg); @@ -1890,7 +1928,7 @@ public class GridCacheProcessor extends GridProcessorAdapter { caches.put(maskNull(cacheCtx.name()), cache); - startCache(cache); + startCache(cache, schema != null ? schema : new QuerySchema()); onKernalStart(cache); } @@ -2137,6 +2175,7 @@ public class GridCacheProcessor extends GridProcessorAdapter { req.cacheType(desc.cacheType()); req.deploymentId(desc.deploymentId()); req.receivedFrom(desc.receivedFrom()); + req.schema(desc.schema()); reqs.add(req); } @@ -2149,6 +2188,7 @@ public class GridCacheProcessor extends GridProcessorAdapter { null); req.startCacheConfiguration(desc.cacheConfiguration()); + req.schema(desc.schema()); req.template(true); @@ -2178,6 +2218,7 @@ public class GridCacheProcessor extends GridProcessorAdapter { req.cacheType(desc.cacheType()); req.deploymentId(desc.deploymentId()); req.receivedFrom(desc.receivedFrom()); + req.schema(desc.schema()); reqs.add(req); @@ -2199,7 +2240,7 @@ public class GridCacheProcessor extends GridProcessorAdapter { onDiscoDataReceived( data.joiningNodeId(), data.joiningNodeId(), - (DynamicCacheChangeBatch) joiningNodeData); + (DynamicCacheChangeBatch) joiningNodeData, true); } } @@ -2212,7 +2253,7 @@ public class GridCacheProcessor extends GridProcessorAdapter { if (e.getValue() != null && e.getValue() instanceof DynamicCacheChangeBatch) { DynamicCacheChangeBatch batch = (DynamicCacheChangeBatch) e.getValue(); - onDiscoDataReceived(data.joiningNodeId(), e.getKey(), batch); + onDiscoDataReceived(data.joiningNodeId(), e.getKey(), batch, false); } } } @@ -2222,8 +2263,9 @@ public class GridCacheProcessor extends GridProcessorAdapter { * @param joiningNodeId Joining node id. * @param rmtNodeId Rmt node id. * @param batch Batch. + * @param join Whether this is data from joining node. */ - private void onDiscoDataReceived(UUID joiningNodeId, UUID rmtNodeId, DynamicCacheChangeBatch batch) { + private void onDiscoDataReceived(UUID joiningNodeId, UUID rmtNodeId, DynamicCacheChangeBatch batch, boolean join) { if (batch.clientReconnect()) { if (ctx.clientDisconnected()) { if (clientReconnectReqs == null) @@ -2253,7 +2295,8 @@ public class GridCacheProcessor extends GridProcessorAdapter { ccfg, req.cacheType(), true, - req.deploymentId()); + req.deploymentId(), + req.schema()); registeredTemplates.put(maskNull(req.cacheName()), desc); } @@ -2275,6 +2318,10 @@ public class GridCacheProcessor extends GridProcessorAdapter { if (existing.locallyConfigured()) { existing.addRemoteConfiguration(rmtNodeId, req.startCacheConfiguration()); + if (!join) + // Overwrite existing with remote. + existing.schema(req.schema()); + ctx.discovery().setCacheFilter( req.cacheName(), ccfg.getNodeFilter(), @@ -2290,7 +2337,8 @@ public class GridCacheProcessor extends GridProcessorAdapter { ccfg, req.cacheType(), false, - req.deploymentId()); + req.deploymentId(), + req.schema()); // Received statically configured cache. if (req.initiatingNodeId() == null) @@ -2791,12 +2839,10 @@ public class GridCacheProcessor extends GridProcessorAdapter { UUID.randomUUID(), cacheName, ctx.localNodeId()); req.startCacheConfiguration(cfg); - req.template(cfg.getName() != null && cfg.getName().endsWith("*")); - req.nearCacheConfiguration(cfg.getNearConfiguration()); - req.deploymentId(IgniteUuid.randomUuid()); + req.schema(new QuerySchema(cfg.getQueryEntities())); if (CU.isUtilityCache(cacheName)) req.cacheType(CacheType.UTILITY); @@ -2923,10 +2969,13 @@ public class GridCacheProcessor extends GridProcessorAdapter { * @param topVer Current topology version. * @return {@code True} if minor topology version should be increased. */ - public boolean onCustomEvent( - DiscoveryCustomMessage msg, - AffinityTopologyVersion topVer - ) { + public boolean onCustomEvent(DiscoveryCustomMessage msg, AffinityTopologyVersion topVer) { + if (msg instanceof SchemaAbstractDiscoveryMessage) { + ctx.query().onDiscovery((SchemaAbstractDiscoveryMessage)msg); + + return false; + } + if (msg instanceof CacheAffinityChangeMessage) return sharedCtx.affinity().onCustomEvent(((CacheAffinityChangeMessage)msg)); @@ -2964,8 +3013,8 @@ public class GridCacheProcessor extends GridProcessorAdapter { DynamicCacheDescriptor desc = registeredTemplates.get(maskNull(req.cacheName())); if (desc == null) { - DynamicCacheDescriptor templateDesc = - new DynamicCacheDescriptor(ctx, ccfg, req.cacheType(), true, req.deploymentId()); + DynamicCacheDescriptor templateDesc = new DynamicCacheDescriptor(ctx, ccfg, req.cacheType(), true, + req.deploymentId(), req.schema()); DynamicCacheDescriptor old = registeredTemplates.put(maskNull(ccfg.getName()), templateDesc); @@ -3008,8 +3057,8 @@ public class GridCacheProcessor extends GridProcessorAdapter { assert req.cacheType() != null : req; assert F.eq(ccfg.getName(), req.cacheName()) : req; - DynamicCacheDescriptor startDesc = - new DynamicCacheDescriptor(ctx, ccfg, req.cacheType(), false, req.deploymentId()); + DynamicCacheDescriptor startDesc = new DynamicCacheDescriptor(ctx, ccfg, req.cacheType(), false, + req.deploymentId(), req.schema()); if (newTopVer == null) { newTopVer = new AffinityTopologyVersion(topVer.topologyVersion(), @@ -3605,6 +3654,7 @@ public class GridCacheProcessor extends GridProcessorAdapter { req.template(true); req.startCacheConfiguration(cfg); + req.schema(new QuerySchema(cfg.getQueryEntities())); req.deploymentId(IgniteUuid.randomUuid()); @@ -3769,16 +3819,28 @@ public class GridCacheProcessor extends GridProcessorAdapter { * * @throws IgniteCheckedException In case of error. */ - public void createMissingCaches() throws IgniteCheckedException { + public void createMissingQueryCaches() throws IgniteCheckedException { for (Map.Entry e : registeredCaches.entrySet()) { - CacheConfiguration ccfg = e.getValue().cacheConfiguration(); + DynamicCacheDescriptor desc = e.getValue(); - if (!caches.containsKey(maskNull(ccfg.getName())) && QueryUtils.isEnabled(ccfg)) - dynamicStartCache(null, ccfg.getName(), null, false, true, true).get(); + if (isMissingQueryCache(desc)) + dynamicStartCache(null, desc.cacheConfiguration().getName(), null, false, true, true).get(); } } /** + * Whether cache defined by provided descriptor is not yet started and has queries enabled. + * + * @param desc Descriptor. + * @return {@code True} if this is missing query cache. + */ + private boolean isMissingQueryCache(DynamicCacheDescriptor desc) { + CacheConfiguration ccfg = desc.cacheConfiguration(); + + return !caches.containsKey(maskNull(ccfg.getName())) && QueryUtils.isEnabled(ccfg); + } + + /** * Registers MBean for cache components. * * @param obj Cache component. @@ -4008,6 +4070,7 @@ public class GridCacheProcessor extends GridProcessorAdapter { req.deploymentId(desc.deploymentId()); req.startCacheConfiguration(descCfg); + req.schema(desc.schema()); } } else { @@ -4020,6 +4083,7 @@ public class GridCacheProcessor extends GridProcessorAdapter { initialize(cfg, cacheObjCtx); req.startCacheConfiguration(cfg); + req.schema(new QuerySchema(cfg.getQueryEntities())); } } else { @@ -4039,6 +4103,7 @@ public class GridCacheProcessor extends GridProcessorAdapter { req.deploymentId(desc.deploymentId()); req.startCacheConfiguration(ccfg); + req.schema(desc.schema()); } if (nearCfg != null) http://git-wip-us.apache.org/repos/asf/ignite/blob/2edb935c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java index ad9eeb1..34bb321 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java @@ -293,6 +293,8 @@ public class GridCacheSharedContext { mgr.start(this); } + kernalCtx.query().onCacheReconnect(); + for (GridCacheSharedManager mgr : mgrs) mgr.onKernalStart(true); } http://git-wip-us.apache.org/repos/asf/ignite/blob/2edb935c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java index f6827ab..654d306 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java @@ -2077,11 +2077,6 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter extends GridCacheManagerAdapte fields.put(type.name(), fieldsMap); - Collection indexesCol = - new ArrayList<>(type.indexes().size()); + Map idxs = type.indexes(); - for (Map.Entry e : type.indexes().entrySet()) { + Collection indexesCol = new ArrayList<>(idxs.size()); + + for (Map.Entry e : idxs.entrySet()) { GridQueryIndexDescriptor desc = e.getValue(); // Add only SQL indexes. http://git-wip-us.apache.org/repos/asf/ignite/blob/2edb935c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/IgniteQueryErrorCode.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/IgniteQueryErrorCode.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/IgniteQueryErrorCode.java index 93b8d47..d05c9fd 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/IgniteQueryErrorCode.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/IgniteQueryErrorCode.java @@ -33,7 +33,7 @@ public final class IgniteQueryErrorCode { /** General parsing error - for the cases when there's no more specific code available. */ public final static int PARSING = 1001; - /** Code encountered unexpected type of SQL operation - like {@code EXPLAIN MERGE}. */ + /** Requested operation is not supported. */ public final static int UNSUPPORTED_OPERATION = 1002; /* 2xxx - analysis errors */ @@ -58,9 +58,24 @@ public final class IgniteQueryErrorCode { /** Statement type does not match that declared by JDBC driver. */ public final static int STMT_TYPE_MISMATCH = 3003; - /** Statement type does not match that declared by JDBC driver. */ + /** DROP TABLE failed. */ public final static int TABLE_DROP_FAILED = 3004; + /** Index already exists. */ + public final static int INDEX_ALREADY_EXISTS = 3005; + + /** Index does not exist. */ + public final static int INDEX_NOT_FOUND = 3006; + + /** Required table already exists. */ + public final static int TABLE_ALREADY_EXISTS = 3007; + + /** Required column not found. */ + public final static int COLUMN_NOT_FOUND = 3008; + + /** Required column already exists. */ + public final static int COLUMN_ALREADY_EXISTS = 3009; + /* 4xxx - cache related runtime errors */ /** Attempt to INSERT a key that is already in cache. */ @@ -78,6 +93,14 @@ public final class IgniteQueryErrorCode { /** {@link EntryProcessor} has thrown an exception during {@link IgniteCache#invokeAll}. */ public final static int ENTRY_PROCESSING = 4005; + /** Cache not found. */ + public final static int CACHE_NOT_FOUND = 4006; + + /** */ + private IgniteQueryErrorCode() { + // No-op. + } + /** * Create a {@link SQLException} for given code and message with null state. * http://git-wip-us.apache.org/repos/asf/ignite/blob/2edb935c/modules/core/src/main/java/org/apache/ignite/internal/processors/pool/PoolProcessor.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/pool/PoolProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/pool/PoolProcessor.java index f6c8b2e..37bbb54 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/pool/PoolProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/pool/PoolProcessor.java @@ -137,6 +137,11 @@ public class PoolProcessor extends GridProcessorAdapter { return ctx.getQueryExecutorService(); + case GridIoPolicy.SCHEMA_POOL: + assert ctx.getSchemaExecutorService() != null : "Query pool is not configured."; + + return ctx.getSchemaExecutorService(); + default: { if (plc < 0) throw new IgniteCheckedException("Policy cannot be negative: " + plc); http://git-wip-us.apache.org/repos/asf/ignite/blob/2edb935c/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexDescriptor.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexDescriptor.java index 7f64dd7..69130fb 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexDescriptor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexDescriptor.java @@ -29,6 +29,11 @@ import java.util.Collection; */ public interface GridQueryIndexDescriptor { /** + * @return Name. + */ + public String name(); + + /** * Gets all fields to be indexed. * * @return Fields to be indexed. http://git-wip-us.apache.org/repos/asf/ignite/blob/2edb935c/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java index d49ea57..7746ba5 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 @@ -24,6 +24,7 @@ import java.util.List; import javax.cache.Cache; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteDataStreamer; +import org.apache.ignite.cache.QueryIndex; import org.apache.ignite.cache.query.QueryCursor; import org.apache.ignite.cache.query.SqlFieldsQuery; import org.apache.ignite.cache.query.SqlQuery; @@ -36,6 +37,7 @@ import org.apache.ignite.internal.processors.cache.KeyCacheObject; import org.apache.ignite.internal.processors.cache.database.CacheDataRow; import org.apache.ignite.internal.processors.cache.database.tree.BPlusTree; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheVisitor; import org.apache.ignite.internal.util.GridSpinBusyLock; import org.apache.ignite.internal.util.lang.GridCloseableIterator; import org.apache.ignite.lang.IgniteBiTuple; @@ -128,13 +130,38 @@ public interface GridQueryIndexing { * * @param spaceName Space name. * @param qry Text query. - * @param type Query return type. + * @param typeName Type name. * @param filter Space name and key filter. * @return Queried rows. * @throws IgniteCheckedException If failed. */ public GridCloseableIterator> queryLocalText(@Nullable String spaceName, String qry, - GridQueryTypeDescriptor type, IndexingQueryFilter filter) throws IgniteCheckedException; + String typeName, IndexingQueryFilter filter) throws IgniteCheckedException; + + /** + * Create new index locally. + * + * @param spaceName Space 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(@Nullable String spaceName, String tblName, QueryIndexDescriptorImpl idxDesc, + boolean ifNotExists, SchemaIndexCacheVisitor cacheVisitor) throws IgniteCheckedException; + + /** + * Remove index from the space. + * + * @param spaceName Space 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(@Nullable String spaceName, String idxName, boolean ifExists) + throws IgniteCheckedException; /** * Registers cache. @@ -169,17 +196,17 @@ public interface GridQueryIndexing { * Unregisters type and removes all corresponding data. * * @param spaceName Space name. - * @param type Type descriptor. + * @param typeName Type name. * @throws IgniteCheckedException If failed. */ - public void unregisterType(@Nullable String spaceName, GridQueryTypeDescriptor type) throws IgniteCheckedException; + public void unregisterType(@Nullable String spaceName, String typeName) throws IgniteCheckedException; /** * Updates index. Note that key is unique for space, so if space contains multiple indexes * the key should be removed from indexes other than one being updated. * * @param spaceName Space name. - * @param type Value type. + * @param typeName Type name. * @param key Key. * @param val Value. * @param ver Version. @@ -187,7 +214,7 @@ public interface GridQueryIndexing { * @throws IgniteCheckedException If failed. */ public void store(@Nullable String spaceName, - GridQueryTypeDescriptor type, + String typeName, KeyCacheObject key, int partId, CacheObject val, @@ -266,11 +293,11 @@ public interface GridQueryIndexing { /** * Prepare native statement to retrieve JDBC metadata from. * - * @param schema Schema. + * @param space Schema. * @param sql Query. * @return {@link PreparedStatement} from underlying engine to supply metadata to Prepared - most likely H2. */ - public PreparedStatement prepareNativeStatement(String schema, String sql) throws SQLException; + public PreparedStatement prepareNativeStatement(String space, String sql) throws SQLException; /** * Gets space name from database schema.