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 06404200C82 for ; Sat, 27 May 2017 14:07:12 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 04C07160BD7; Sat, 27 May 2017 12:07:12 +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 A719A160BC6 for ; Sat, 27 May 2017 14:07:10 +0200 (CEST) Received: (qmail 19023 invoked by uid 500); 27 May 2017 12:07:09 -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 19014 invoked by uid 99); 27 May 2017 12:07: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; Sat, 27 May 2017 12:07:09 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id BF34CDFB01; Sat, 27 May 2017 12:07:09 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: vozerov@apache.org To: commits@ignite.apache.org Message-Id: X-Mailer: ASF-Git Admin Mailer Subject: ignite git commit: IGNITE-5304: SQL: decoupled DDL from schema. This closes #2017. Date: Sat, 27 May 2017 12:07:09 +0000 (UTC) archived-at: Sat, 27 May 2017 12:07:12 -0000 Repository: ignite Updated Branches: refs/heads/master 427ca6900 -> 40851c7ff IGNITE-5304: SQL: decoupled DDL from schema. This closes #2017. Project: http://git-wip-us.apache.org/repos/asf/ignite/repo Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/40851c7f Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/40851c7f Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/40851c7f Branch: refs/heads/master Commit: 40851c7ff44c0ec97d9cca53923c8db0a38f78cd Parents: 427ca69 Author: devozerov Authored: Sat May 27 15:07:01 2017 +0300 Committer: devozerov Committed: Sat May 27 15:07:01 2017 +0300 ---------------------------------------------------------------------- .../processors/query/GridQueryIndexing.java | 9 ++- .../processors/query/GridQueryProcessor.java | 32 +++++----- .../processors/query/QueryIndexKey.java | 28 +++------ .../processors/query/h2/IgniteH2Indexing.java | 32 +++++----- .../query/h2/ddl/DdlStatementsProcessor.java | 66 ++++++++++++-------- .../DynamicIndexAbstractConcurrentSelfTest.java | 8 +-- 6 files changed, 83 insertions(+), 92 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/40851c7f/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 77c581c..9d66c0a 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 @@ -142,27 +142,26 @@ public interface GridQueryIndexing { /** * Create new index locally. * - * @param cacheName Cache name. + * @param schemaName Schema 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 cacheName, String tblName, QueryIndexDescriptorImpl idxDesc, + public void dynamicIndexCreate(String schemaName, String tblName, QueryIndexDescriptorImpl idxDesc, boolean ifNotExists, SchemaIndexCacheVisitor cacheVisitor) throws IgniteCheckedException; /** * Remove index from the cache. * - * @param cacheName cache name. + * @param schemaName Schema 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 cacheName, String idxName, boolean ifExists) - throws IgniteCheckedException; + public void dynamicIndexDrop(String schemaName, String idxName, boolean ifExists) throws IgniteCheckedException; /** * Registers cache. http://git-wip-us.apache.org/repos/asf/ignite/blob/40851c7f/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java index 3233fd2..76cde17 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 @@ -933,7 +933,7 @@ public class GridQueryProcessor extends GridProcessorAdapter { if (err == null) { String idxName = op0.index().getName(); - QueryIndexKey idxKey = new QueryIndexKey(cacheName, idxName); + QueryIndexKey idxKey = new QueryIndexKey(op.schemaName(), idxName); if (idxs.get(idxKey) != null) { if (op0.ifNotExists()) @@ -948,7 +948,7 @@ public class GridQueryProcessor extends GridProcessorAdapter { String idxName = op0.indexName(); - QueryIndexDescriptorImpl oldIdx = idxs.get(new QueryIndexKey(cacheName, idxName)); + QueryIndexDescriptorImpl oldIdx = idxs.get(new QueryIndexKey(op.schemaName(), idxName)); if (oldIdx == null) { if (op0.ifExists()) @@ -1159,7 +1159,7 @@ public class GridQueryProcessor extends GridProcessorAdapter { if (log.isDebugEnabled()) log.debug("Local operation finished successfully [opId=" + op.id() + ']'); - String cacheName = op.cacheName(); + String schemaName = op.schemaName(); try { if (op instanceof SchemaIndexCreateOperation) { @@ -1169,7 +1169,7 @@ public class GridQueryProcessor extends GridProcessorAdapter { QueryIndexDescriptorImpl idxDesc = type.index(op0.indexName()); - QueryIndexKey idxKey = new QueryIndexKey(cacheName, op0.indexName()); + QueryIndexKey idxKey = new QueryIndexKey(schemaName, op0.indexName()); idxs.put(idxKey, idxDesc); } @@ -1180,7 +1180,7 @@ public class GridQueryProcessor extends GridProcessorAdapter { QueryUtils.processDynamicIndexChange(op0.indexName(), null, type); - QueryIndexKey idxKey = new QueryIndexKey(cacheName, op0.indexName()); + QueryIndexKey idxKey = new QueryIndexKey(schemaName, op0.indexName()); idxs.remove(idxKey); } @@ -1251,12 +1251,12 @@ public class GridQueryProcessor extends GridProcessorAdapter { SchemaIndexCacheVisitor visitor = new SchemaIndexCacheVisitorImpl(this, cache.context(), cacheName, op0.tableName(), cancelTok); - idx.dynamicIndexCreate(cacheName, op0.tableName(), idxDesc, op0.ifNotExists(), visitor); + idx.dynamicIndexCreate(op0.schemaName(), op0.tableName(), idxDesc, op0.ifNotExists(), visitor); } else if (op instanceof SchemaIndexDropOperation) { SchemaIndexDropOperation op0 = (SchemaIndexDropOperation) op; - idx.dynamicIndexDrop(cacheName, op0.indexName(), op0.ifExists()); + idx.dynamicIndexDrop(op0.schemaName(), op0.indexName(), op0.ifExists()); } else throw new SchemaOperationException("Unsupported operation: " + op); @@ -1293,10 +1293,9 @@ public class GridQueryProcessor extends GridProcessorAdapter { CacheConfiguration newCfg = new CacheConfiguration<>(templateCfg); newCfg.setName(entity.getTableName()); - newCfg.setQueryEntities(Collections.singleton(entity)); - // We want to preserve user specified names as they are + // Preserve user specified names as they are. newCfg.setSqlEscapeAll(true); boolean res = ctx.grid().getOrCreateCache0(newCfg).get2(); @@ -1308,14 +1307,14 @@ public class GridQueryProcessor extends GridProcessorAdapter { /** * Drop table by destroying its cache if it's an 1:1 per cache table. * - * @param schemaName Schema name. + * @param cacheName Cache name. * @param tblName Table name. * @param ifExists Quietly ignore this command if table does not exist. * @throws SchemaOperationException if {@code ifExists} is {@code false} and cache was not found. */ @SuppressWarnings("unchecked") - public void dynamicTableDrop(String schemaName, String tblName, boolean ifExists) throws SchemaOperationException { - boolean res = ctx.grid().destroyCache0(tblName); + public void dynamicTableDrop(String cacheName, String tblName, boolean ifExists) throws SchemaOperationException { + boolean res = ctx.grid().destroyCache0(cacheName); if (!res && !ifExists) throw new SchemaOperationException(SchemaOperationException.CODE_TABLE_NOT_FOUND, tblName); @@ -1352,13 +1351,14 @@ public class GridQueryProcessor extends GridProcessorAdapter { types.put(altTypeId, desc); for (QueryIndexDescriptorImpl idx : desc.indexes0()) { - QueryIndexKey idxKey = new QueryIndexKey(cacheName, idx.name()); + QueryIndexKey idxKey = new QueryIndexKey(schemaName, idx.name()); QueryIndexDescriptorImpl oldIdx = idxs.putIfAbsent(idxKey, idx); if (oldIdx != null) { throw new IgniteException("Duplicate index name [cache=" + cacheName + - ", idxName=" + idx.name() + ", existingTable=" + oldIdx.typeDescriptor().tableName() + + ", schemaName=" + schemaName + ", idxName=" + idx.name() + + ", existingTable=" + oldIdx.typeDescriptor().tableName() + ", table=" + desc.tableName() + ']'); } } @@ -1409,9 +1409,7 @@ public class GridQueryProcessor extends GridProcessorAdapter { while (idxIt.hasNext()) { Map.Entry idxEntry = idxIt.next(); - QueryIndexKey idxKey = idxEntry.getKey(); - - if (F.eq(cacheName, idxKey.cacheName())) + if (F.eq(cacheName, idxEntry.getValue().typeDescriptor().cacheName())) idxIt.remove(); } http://git-wip-us.apache.org/repos/asf/ignite/blob/40851c7f/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryIndexKey.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryIndexKey.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryIndexKey.java index 7fdb805..fea6594 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; - /** Cache name. */ - private final String cacheName; + /** Schema name. */ + private final String schemaName; /** Name. */ private final String name; @@ -38,31 +38,17 @@ public class QueryIndexKey implements Serializable { /** * Constructor. * - * @param cacheName Cache name. + * @param schemaName Schema name. * @param name Name. */ - public QueryIndexKey(String cacheName, String name) { - this.cacheName = cacheName; + public QueryIndexKey(String schemaName, String name) { + this.schemaName = schemaName; this.name = name; } - /** - * @return Cache name. - */ - public String cacheName() { - return cacheName; - } - - /** - * @return Name. - */ - public String name() { - return name; - } - /** {@inheritDoc} */ @Override public int hashCode() { - return 31 * (cacheName != null ? cacheName.hashCode() : 0) + (name != null ? name.hashCode() : 0); + return 31 * (schemaName != null ? schemaName.hashCode() : 0) + (name != null ? name.hashCode() : 0); } /** {@inheritDoc} */ @@ -75,7 +61,7 @@ public class QueryIndexKey implements Serializable { QueryIndexKey other = (QueryIndexKey)o; - return F.eq(name, other.name) && F.eq(cacheName, other.cacheName); + return F.eq(name, other.name) && F.eq(schemaName, other.schemaName); } /** {@inheritDoc} */ http://git-wip-us.apache.org/repos/asf/ignite/blob/40851c7f/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java index 71d3ad5..0291b0a 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 @@ -670,12 +670,12 @@ public class IgniteH2Indexing implements GridQueryIndexing { /** * Add initial user index. * - * @param cacheName Cache name. + * @param schemaName Schema name. * @param desc Table descriptor. * @param h2Idx User index. * @throws IgniteCheckedException If failed. */ - private void addInitialUserIndex(String cacheName, H2TableDescriptor desc, GridH2IndexBase h2Idx) + private void addInitialUserIndex(String schemaName, H2TableDescriptor desc, GridH2IndexBase h2Idx) throws IgniteCheckedException { GridH2Table h2Tbl = desc.table(); @@ -684,7 +684,7 @@ public class IgniteH2Indexing implements GridQueryIndexing { try { String sql = H2Utils.indexCreateSql(desc.fullTableName(), h2Idx, false); - executeSql(cacheName, sql); + executeSql(schemaName, sql); } catch (Exception e) { // Rollback and re-throw. @@ -695,12 +695,10 @@ public class IgniteH2Indexing implements GridQueryIndexing { } /** {@inheritDoc} */ - @Override public void dynamicIndexCreate(final String cacheName, final String tblName, + @Override public void dynamicIndexCreate(final String schemaName, final String tblName, final QueryIndexDescriptorImpl idxDesc, boolean ifNotExists, SchemaIndexCacheVisitor cacheVisitor) throws IgniteCheckedException { // Locate table. - String schemaName = schema(cacheName); - H2Schema schema = schemas.get(schemaName); H2TableDescriptor desc = (schema != null ? schema.tableByName(tblName) : null); @@ -740,7 +738,7 @@ public class IgniteH2Indexing implements GridQueryIndexing { // prepared statements are re-built. String sql = H2Utils.indexCreateSql(desc.fullTableName(), h2Idx, ifNotExists); - executeSql(cacheName, sql); + executeSql(schemaName, sql); } catch (Exception e) { // Rollback and re-throw. @@ -752,25 +750,23 @@ public class IgniteH2Indexing implements GridQueryIndexing { /** {@inheritDoc} */ @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter") - @Override public void dynamicIndexDrop(final String cacheName, String idxName, boolean ifExists) + @Override public void dynamicIndexDrop(final String schemaName, String idxName, boolean ifExists) throws IgniteCheckedException{ - String schemaName = schema(cacheName); - String sql = H2Utils.indexDropSql(schemaName, idxName, ifExists); - executeSql(cacheName, sql); + executeSql(schemaName, sql); } /** * Execute DDL command. * - * @param cacheName Cache name. + * @param schemaName Schema name. * @param sql SQL. * @throws IgniteCheckedException If failed. */ - private void executeSql(String cacheName, String sql) throws IgniteCheckedException { + private void executeSql(String schemaName, String sql) throws IgniteCheckedException { try { - Connection conn = connectionForCache(cacheName); + Connection conn = connectionForSchema(schemaName); try (PreparedStatement stmt = prepareStatement(conn, sql, false)) { stmt.execute(); @@ -1609,7 +1605,7 @@ public class IgniteH2Indexing implements GridQueryIndexing { try { Connection conn = connectionForThread(schemaName); - createTable(cacheName, schema, tbl, conn); + createTable(schemaName, schema, tbl, conn); schema.add(tbl); } @@ -1653,14 +1649,14 @@ public class IgniteH2Indexing implements GridQueryIndexing { /** * Create db table by using given table descriptor. * - * @param cacheName Cache name. + * @param schemaName Schema 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 cacheName, H2Schema schema, H2TableDescriptor tbl, Connection conn) + private void createTable(String schemaName, H2Schema schema, H2TableDescriptor tbl, Connection conn) throws SQLException, IgniteCheckedException { assert schema != null; assert tbl != null; @@ -1693,7 +1689,7 @@ public class IgniteH2Indexing implements GridQueryIndexing { GridH2Table h2Tbl = H2TableEngine.createTable(conn, sql.toString(), rowDesc, rowFactory, tbl); for (GridH2IndexBase usrIdx : tbl.createUserIndexes()) - addInitialUserIndex(cacheName, tbl, usrIdx); + addInitialUserIndex(schemaName, tbl, usrIdx); if (dataTables.putIfAbsent(h2Tbl.identifier(), h2Tbl) != null) throw new IllegalStateException("Table already exists: " + h2Tbl.identifierString()); http://git-wip-us.apache.org/repos/asf/ignite/blob/40851c7f/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java index 7157d1f..e324ed9 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java @@ -88,34 +88,34 @@ public class DdlStatementsProcessor { throws IgniteCheckedException { assert stmt instanceof JdbcPreparedStatement; - IgniteInternalFuture fut; + IgniteInternalFuture fut = null; try { - GridSqlStatement gridStmt = new GridSqlQueryParser(false).parse(GridSqlQueryParser.prepared(stmt)); + GridSqlStatement stmt0 = new GridSqlQueryParser(false).parse(GridSqlQueryParser.prepared(stmt)); - if (gridStmt instanceof GridSqlCreateIndex) { - GridSqlCreateIndex createIdx = (GridSqlCreateIndex)gridStmt; + if (stmt0 instanceof GridSqlCreateIndex) { + GridSqlCreateIndex cmd = (GridSqlCreateIndex)stmt0; QueryIndex newIdx = new QueryIndex(); - newIdx.setName(createIdx.index().getName()); + newIdx.setName(cmd.index().getName()); - newIdx.setIndexType(createIdx.index().getIndexType()); + newIdx.setIndexType(cmd.index().getIndexType()); LinkedHashMap flds = new LinkedHashMap<>(); - GridH2Table tbl = idx.dataTable(createIdx.schemaName(), createIdx.tableName()); + GridH2Table tbl = idx.dataTable(cmd.schemaName(), cmd.tableName()); if (tbl == null) throw new SchemaOperationException(SchemaOperationException.CODE_TABLE_NOT_FOUND, - createIdx.tableName()); + cmd.tableName()); assert tbl.rowDescriptor() != null; // Let's replace H2's table and property names by those operated by GridQueryProcessor. GridQueryTypeDescriptor typeDesc = tbl.rowDescriptor().type(); - for (Map.Entry e : createIdx.index().getFields().entrySet()) { + for (Map.Entry e : cmd.index().getFields().entrySet()) { GridQueryProperty prop = typeDesc.property(e.getKey()); if (prop == null) @@ -126,39 +126,51 @@ public class DdlStatementsProcessor { newIdx.setFields(flds); - fut = ctx.query().dynamicIndexCreate(tbl.cacheName(), createIdx.schemaName(), typeDesc.tableName(), - newIdx, createIdx.ifNotExists()); + fut = ctx.query().dynamicIndexCreate(tbl.cacheName(), cmd.schemaName(), typeDesc.tableName(), + newIdx, cmd.ifNotExists()); } - else if (gridStmt instanceof GridSqlDropIndex) { - GridSqlDropIndex dropIdx = (GridSqlDropIndex)gridStmt; + else if (stmt0 instanceof GridSqlDropIndex) { + GridSqlDropIndex cmd = (GridSqlDropIndex)stmt0; - GridH2Table tbl = idx.dataTableForIndex(dropIdx.schemaName(), dropIdx.indexName()); + GridH2Table tbl = idx.dataTableForIndex(cmd.schemaName(), cmd.indexName()); if (tbl != null) - fut = ctx.query().dynamicIndexDrop(tbl.cacheName(), dropIdx.schemaName(), dropIdx.indexName(), - dropIdx.ifExists()); + fut = ctx.query().dynamicIndexDrop(tbl.cacheName(), cmd.schemaName(), cmd.indexName(), + cmd.ifExists()); else { - if (dropIdx.ifExists()) + if (cmd.ifExists()) fut = new GridFinishedFuture(); else throw new SchemaOperationException(SchemaOperationException.CODE_INDEX_NOT_FOUND, - dropIdx.indexName()); + cmd.indexName()); } } - else if (gridStmt instanceof GridSqlCreateTable) { - GridSqlCreateTable createTbl = (GridSqlCreateTable)gridStmt; + else if (stmt0 instanceof GridSqlCreateTable) { + GridSqlCreateTable cmd = (GridSqlCreateTable)stmt0; - ctx.query().dynamicTableCreate(createTbl.schemaName(), toQueryEntity(createTbl), - createTbl.templateCacheName(), createTbl.ifNotExists()); + GridH2Table tbl = idx.dataTable(cmd.schemaName(), cmd.tableName()); - fut = null; + if (tbl != null) { + if (!cmd.ifNotExists()) + throw new SchemaOperationException(SchemaOperationException.CODE_TABLE_EXISTS, + cmd.tableName()); + } + else + ctx.query().dynamicTableCreate(cmd.schemaName(), toQueryEntity(cmd), cmd.templateCacheName(), + cmd.ifNotExists()); } - else if (gridStmt instanceof GridSqlDropTable) { - GridSqlDropTable dropTbl = (GridSqlDropTable)gridStmt; + else if (stmt0 instanceof GridSqlDropTable) { + GridSqlDropTable cmd = (GridSqlDropTable)stmt0; - ctx.query().dynamicTableDrop(dropTbl.schemaName(), dropTbl.tableName(), dropTbl.ifExists()); + GridH2Table tbl = idx.dataTable(cmd.schemaName(), cmd.tableName()); - fut = null; + if (tbl == null) { + if (!cmd.ifExists()) + throw new SchemaOperationException(SchemaOperationException.CODE_TABLE_NOT_FOUND, + cmd.tableName()); + } + else + ctx.query().dynamicTableDrop(tbl.cacheName(), cmd.tableName(), cmd.ifExists()); } else throw new IgniteSQLException("Unsupported DDL operation: " + sql, http://git-wip-us.apache.org/repos/asf/ignite/blob/40851c7f/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractConcurrentSelfTest.java ---------------------------------------------------------------------- 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 5d93c12..9936a81 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 @@ -1043,20 +1043,20 @@ public abstract class DynamicIndexAbstractConcurrentSelfTest extends DynamicInde */ private static class BlockingIndexing extends IgniteH2Indexing { /** {@inheritDoc} */ - @Override public void dynamicIndexCreate(@NotNull String cacheName, String tblName, + @Override public void dynamicIndexCreate(@NotNull String schemaName, String tblName, QueryIndexDescriptorImpl idxDesc, boolean ifNotExists, SchemaIndexCacheVisitor cacheVisitor) throws IgniteCheckedException { awaitIndexing(ctx.localNodeId()); - super.dynamicIndexCreate(cacheName, tblName, idxDesc, ifNotExists, cacheVisitor); + super.dynamicIndexCreate(schemaName, tblName, idxDesc, ifNotExists, cacheVisitor); } /** {@inheritDoc} */ - @Override public void dynamicIndexDrop(@NotNull String cacheName, String idxName, boolean ifExists) + @Override public void dynamicIndexDrop(@NotNull String schemaName, String idxName, boolean ifExists) throws IgniteCheckedException{ awaitIndexing(ctx.localNodeId()); - super.dynamicIndexDrop(cacheName, idxName, ifExists); + super.dynamicIndexDrop(schemaName, idxName, ifExists); } } }