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 19249200D1A for ; Mon, 9 Oct 2017 16:48:03 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 175711609BB; Mon, 9 Oct 2017 14:48:03 +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 1FA46160BFA for ; Mon, 9 Oct 2017 16:47:58 +0200 (CEST) Received: (qmail 87388 invoked by uid 500); 9 Oct 2017 14:47:58 -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 87001 invoked by uid 99); 9 Oct 2017 14:47:58 -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; Mon, 09 Oct 2017 14:47:58 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 2A228F5D4E; Mon, 9 Oct 2017 14:47:57 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: agura@apache.org To: commits@ignite.apache.org Date: Mon, 09 Oct 2017 14:48:35 -0000 Message-Id: <168d9a3ac1d24fbe9938a63c0b1b0cf1@git.apache.org> In-Reply-To: References: X-Mailer: ASF-Git Admin Mailer Subject: [40/50] [abbrv] ignite git commit: IGNITE-6054: SQL: implemented "WRAP_KEY" and "WRAP_VALUE" modes for CREATE TABLE. This closes #2784. archived-at: Mon, 09 Oct 2017 14:48:03 -0000 IGNITE-6054: SQL: implemented "WRAP_KEY" and "WRAP_VALUE" modes for CREATE TABLE. This closes #2784. Project: http://git-wip-us.apache.org/repos/asf/ignite/repo Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/79d94cf4 Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/79d94cf4 Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/79d94cf4 Branch: refs/heads/ignite-6305 Commit: 79d94cf4bbc906449a4c6995801fa61daaf80faa Parents: 74f0400 Author: Alexander Paschenko Authored: Fri Oct 6 18:04:44 2017 +0300 Committer: devozerov Committed: Fri Oct 6 18:04:44 2017 +0300 ---------------------------------------------------------------------- .../jdbc/thin/JdbcThinMetadataSelfTest.java | 2 +- .../cache/query/IgniteQueryErrorCode.java | 2 +- .../processors/query/GridQueryIndexing.java | 22 +- .../processors/query/GridQueryProcessor.java | 8 +- .../query/QueryTypeDescriptorImpl.java | 42 ++- .../query/property/QueryBinaryProperty.java | 1 - ...IgniteClientCacheInitializationFailTest.java | 18 +- .../query/h2/DmlStatementsProcessor.java | 26 +- .../internal/processors/query/h2/H2Schema.java | 17 +- .../internal/processors/query/h2/H2TypeKey.java | 64 ++++ .../processors/query/h2/IgniteH2Indexing.java | 33 +- .../query/h2/ddl/DdlStatementsProcessor.java | 36 ++- .../query/h2/sql/GridSqlCreateTable.java | 34 +++ .../query/h2/sql/GridSqlQueryParser.java | 58 +++- ...ynamicColumnsAbstractConcurrentSelfTest.java | 57 ++-- ...umnsConcurrentAtomicPartitionedSelfTest.java | 2 +- ...lumnsConcurrentAtomicReplicatedSelfTest.java | 2 +- ...currentTransactionalPartitionedSelfTest.java | 2 +- ...ncurrentTransactionalReplicatedSelfTest.java | 5 +- .../H2DynamicColumnsAbstractBasicSelfTest.java | 44 +++ .../cache/index/H2DynamicTableSelfTest.java | 301 +++++++++++++++++-- .../query/IgniteSqlNotNullConstraintTest.java | 2 +- .../h2/GridIndexingSpiAbstractSelfTest.java | 47 +-- .../Cache/Query/CacheDmlQueriesTest.cs | 4 +- 24 files changed, 673 insertions(+), 156 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/79d94cf4/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinMetadataSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinMetadataSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinMetadataSelfTest.java index 6c20de0..4e1ae4d 100644 --- a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinMetadataSelfTest.java +++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinMetadataSelfTest.java @@ -139,7 +139,7 @@ public class JdbcThinMetadataSelfTest extends JdbcThinAbstractSelfTest { Statement stmt = conn.createStatement(); stmt.execute("CREATE TABLE TEST (ID INT, NAME VARCHAR(50), VAL VARCHAR(50), PRIMARY KEY (ID, NAME))"); - stmt.execute("CREATE TABLE \"Quoted\" (\"Id\" INT primary key, \"Name\" VARCHAR(50))"); + stmt.execute("CREATE TABLE \"Quoted\" (\"Id\" INT primary key, \"Name\" VARCHAR(50)) WITH WRAP_KEY"); stmt.execute("CREATE INDEX \"MyTestIndex quoted\" on \"Quoted\" (\"Id\" DESC)"); stmt.execute("CREATE INDEX IDX ON TEST (ID ASC)"); } http://git-wip-us.apache.org/repos/asf/ignite/blob/79d94cf4/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 8e5af31..e0ff9a4 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 @@ -91,7 +91,7 @@ public final class IgniteQueryErrorCode { /** Attempt to INSERT or MERGE {@code null} key. */ public final static int NULL_KEY = 4003; - /** Attempt to INSERT or MERGE {@code null} value. */ + /** Attempt to INSERT or MERGE {@code null} value, or to to set {@code null} to a {@code NOT NULL} column. */ public final static int NULL_VALUE = 4004; /** {@link EntryProcessor} has thrown an exception during {@link IgniteCache#invokeAll}. */ http://git-wip-us.apache.org/repos/asf/ignite/blob/79d94cf4/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 b8445ca..93d541d 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 @@ -66,13 +66,13 @@ public interface GridQueryIndexing { * Parses SQL query into two step query and executes it. * * @param schemaName Schema name. + * @param cacheName Cache name. * @param qry Query. * @param keepBinary Keep binary flag. - * @param mainCacheId Main cache ID. - * @return Cursor. + * @param mainCacheId Main cache ID. @return Cursor. * @throws IgniteCheckedException If failed. */ - public QueryCursor> queryDistributedSql(String schemaName, SqlQuery qry, + public QueryCursor> queryDistributedSql(String schemaName, String cacheName, SqlQuery qry, boolean keepBinary, int mainCacheId) throws IgniteCheckedException; /** @@ -109,12 +109,12 @@ public interface GridQueryIndexing { * Executes regular query. * * @param schemaName Schema name. - * @param qry Query. + * @param cacheName Cache name. + *@param qry Query. * @param filter Cache name and key filter. - * @param keepBinary Keep binary flag. - * @return Cursor. + * @param keepBinary Keep binary flag. @return Cursor. */ - public QueryCursor> queryLocalSql(String schemaName, SqlQuery qry, + public QueryCursor> queryLocalSql(String schemaName, String cacheName, SqlQuery qry, IndexingQueryFilter filter, boolean keepBinary) throws IgniteCheckedException; /** @@ -134,14 +134,14 @@ public interface GridQueryIndexing { * Executes text query. * * @param schemaName Schema name. + * @param cacheName Cache name. * @param qry Text query. * @param typeName Type name. - * @param filter Cache name and key filter. - * @return Queried rows. + * @param filter Cache name and key filter. @return Queried rows. * @throws IgniteCheckedException If failed. */ - public GridCloseableIterator> queryLocalText(String schemaName, String qry, - String typeName, IndexingQueryFilter filter) throws IgniteCheckedException; + public GridCloseableIterator> queryLocalText(String schemaName, String cacheName, + String qry, String typeName, IndexingQueryFilter filter) throws IgniteCheckedException; /** * Create new index locally. http://git-wip-us.apache.org/repos/asf/ignite/blob/79d94cf4/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 56e8a42..f044c1d 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 @@ -2044,7 +2044,7 @@ public class GridQueryProcessor extends GridProcessorAdapter { return executeQuery(GridCacheQueryType.SQL, qry.getSql(), cctx, new IgniteOutClosureX>>() { @Override public QueryCursor> applyx() throws IgniteCheckedException { - return idx.queryDistributedSql(schemaName, qry, keepBinary, mainCacheId); + return idx.queryDistributedSql(schemaName, cctx.name(), qry, keepBinary, mainCacheId); } }, true); } @@ -2088,10 +2088,10 @@ public class GridQueryProcessor extends GridProcessorAdapter { if (cctx.config().getQueryParallelism() > 1) { qry.setDistributedJoins(true); - return idx.queryDistributedSql(schemaName, qry, keepBinary, mainCacheId); + return idx.queryDistributedSql(schemaName, cctx.name(), qry, keepBinary, mainCacheId); } else - return idx.queryLocalSql(schemaName, qry, idx.backupFilter(requestTopVer.get(), + return idx.queryLocalSql(schemaName, cctx.name(), qry, idx.backupFilter(requestTopVer.get(), qry.getPartitions()), keepBinary); } }, true); @@ -2344,7 +2344,7 @@ public class GridQueryProcessor extends GridProcessorAdapter { String typeName = typeName(cacheName, resType); String schemaName = idx.schema(cacheName); - return idx.queryLocalText(schemaName, clause, typeName, filters); + return idx.queryLocalText(schemaName, cacheName, clause, typeName, filters); } }, true); } http://git-wip-us.apache.org/repos/asf/ignite/blob/79d94cf4/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeDescriptorImpl.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeDescriptorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeDescriptorImpl.java index e12476a..72adefd 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 @@ -24,12 +24,12 @@ import java.util.HashMap; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; - import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.cache.QueryIndexType; import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode; import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.A; import org.apache.ignite.internal.util.typedef.internal.S; import org.jetbrains.annotations.Nullable; @@ -453,7 +453,7 @@ public class QueryTypeDescriptorImpl implements GridQueryTypeDescriptor { * Sets key field name. * @param keyFieldName Key field name. */ - public void keyFieldName(String keyFieldName) { + void keyFieldName(String keyFieldName) { this.keyFieldName = keyFieldName; } @@ -464,10 +464,10 @@ public class QueryTypeDescriptorImpl implements GridQueryTypeDescriptor { /** * Sets value field name. - * @param valueFieldName value field name. + * @param valFieldName value field name. */ - public void valueFieldName(String valueFieldName) { - this.valFieldName = valueFieldName; + void valueFieldName(String valFieldName) { + this.valFieldName = valFieldName; } /** {@inheritDoc} */ @@ -480,6 +480,7 @@ public class QueryTypeDescriptorImpl implements GridQueryTypeDescriptor { return keyFieldName != null ? aliases.get(keyFieldName) : null; } + /** {@inheritDoc} */ @Nullable @Override public String valueFieldAlias() { return valFieldName != null ? aliases.get(valFieldName) : null; } @@ -487,17 +488,34 @@ public class QueryTypeDescriptorImpl implements GridQueryTypeDescriptor { /** {@inheritDoc} */ @SuppressWarnings("ForLoopReplaceableByForEach") @Override public void validateKeyAndValue(Object key, Object val) throws IgniteCheckedException { - if (validateProps == null) + if (F.isEmpty(validateProps)) return; - final int size = validateProps.size(); + for (int i = 0; i < validateProps.size(); ++i) { + GridQueryProperty prop = validateProps.get(i); + + Object propVal; + + int errCode; + + if (F.eq(prop.name(), keyFieldName)) { + propVal = key; + + errCode = IgniteQueryErrorCode.NULL_KEY; + } + else if (F.eq(prop.name(), valFieldName)) { + propVal = val; + + errCode = IgniteQueryErrorCode.NULL_VALUE; + } + else { + propVal = prop.value(key, val); - for (int idx = 0; idx < size; ++idx) { - GridQueryProperty prop = validateProps.get(idx); + errCode = IgniteQueryErrorCode.NULL_VALUE; + } - if (prop.value(key, val) == null) - throw new IgniteSQLException("Null value is not allowed for field '" + prop.name() + "'", - IgniteQueryErrorCode.NULL_VALUE); + if (propVal == null) + throw new IgniteSQLException("Null value is not allowed for column '" + prop.name() + "'", errCode); } } } http://git-wip-us.apache.org/repos/asf/ignite/blob/79d94cf4/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryBinaryProperty.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryBinaryProperty.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryBinaryProperty.java index 5d90a43..18508a8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryBinaryProperty.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryBinaryProperty.java @@ -81,7 +81,6 @@ public class QueryBinaryProperty implements GridQueryProperty { */ public QueryBinaryProperty(GridKernalContext ctx, String propName, QueryBinaryProperty parent, Class type, @Nullable Boolean key, String alias, boolean notNull) { - this.ctx = ctx; log = ctx.log(QueryBinaryProperty.class); http://git-wip-us.apache.org/repos/asf/ignite/blob/79d94cf4/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java index 1ebf556..83dd9c9 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java @@ -193,7 +193,7 @@ public class IgniteClientCacheInitializationFailTest extends GridCommonAbstractT /** * @param client Client. */ - @SuppressWarnings("ThrowableNotThrown") + @SuppressWarnings({"ThrowableNotThrown", "ThrowableResultOfMethodCallIgnored"}) private void checkFailedCache(final Ignite client, final String cacheName) { GridTestUtils.assertThrows(log, new Callable() { @Override public Object call() throws Exception { @@ -236,14 +236,14 @@ public class IgniteClientCacheInitializationFailTest extends GridCommonAbstractT } /** {@inheritDoc} */ - @Override public QueryCursor> queryDistributedSql(String schemaName, SqlQuery qry, - boolean keepBinary, int mainCacheId) throws IgniteCheckedException { + @Override public QueryCursor> queryDistributedSql(String schemaName, String cacheName, + SqlQuery qry, boolean keepBinary, int mainCacheId) throws IgniteCheckedException { return null; } /** {@inheritDoc} */ - @Override public List>> queryDistributedSqlFields(String schemaName, SqlFieldsQuery qry, - boolean keepBinary, GridQueryCancel cancel, + @Override public List>> queryDistributedSqlFields(String schemaName, + SqlFieldsQuery qry, boolean keepBinary, GridQueryCancel cancel, @Nullable Integer mainCacheId, boolean failOnMultipleStmts) throws IgniteCheckedException { return null; } @@ -255,8 +255,8 @@ public class IgniteClientCacheInitializationFailTest extends GridCommonAbstractT } /** {@inheritDoc} */ - @Override public QueryCursor> queryLocalSql(String schemaName, SqlQuery qry, - IndexingQueryFilter filter, boolean keepBinary) throws IgniteCheckedException { + @Override public QueryCursor> queryLocalSql(String schemaName, String cacheName, + SqlQuery qry, IndexingQueryFilter filter, boolean keepBinary) throws IgniteCheckedException { return null; } @@ -267,8 +267,8 @@ public class IgniteClientCacheInitializationFailTest extends GridCommonAbstractT } /** {@inheritDoc} */ - @Override public GridCloseableIterator> queryLocalText(String spaceName, String qry, - String typeName, IndexingQueryFilter filter) throws IgniteCheckedException { + @Override public GridCloseableIterator> queryLocalText(String spaceName, + String cacheName, String qry, String typeName, IndexingQueryFilter filter) throws IgniteCheckedException { return null; } http://git-wip-us.apache.org/repos/asf/ignite/blob/79d94cf4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java index ee1875f..98117b2 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 @@ -49,13 +49,13 @@ import org.apache.ignite.binary.BinaryObjectBuilder; import org.apache.ignite.cache.query.SqlFieldsQuery; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.internal.GridKernalContext; -import org.apache.ignite.internal.processors.odbc.SqlStateCode; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.CacheOperationContext; import org.apache.ignite.internal.processors.cache.GridCacheAdapter; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.QueryCursorImpl; import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode; +import org.apache.ignite.internal.processors.odbc.SqlStateCode; import org.apache.ignite.internal.processors.query.GridQueryCacheObjectsIterator; import org.apache.ignite.internal.processors.query.GridQueryCancel; import org.apache.ignite.internal.processors.query.GridQueryFieldsResult; @@ -255,13 +255,12 @@ public class DmlStatementsProcessor { } /** - * Perform given statement against given data streamer. Only rows based INSERT and MERGE are supported - * as well as key bound UPDATE and DELETE (ones with filter {@code WHERE _key = ?}). + * Perform given statement against given data streamer. Only rows based INSERT is supported. * * @param streamer Streamer to feed data to. * @param stmt Statement. * @param args Statement arguments. - * @return Number of rows in given statement for INSERT and MERGE, {@code 1} otherwise. + * @return Number of rows in given INSERT statement. * @throws IgniteCheckedException if failed. */ @SuppressWarnings({"unchecked", "ConstantConditions"}) @@ -916,11 +915,22 @@ public class DmlStatementsProcessor { val = convert(val, rowDesc, desc.valueClass(), plan.colTypes[plan.valColIdx]); } - if (key == null) - throw new IgniteSQLException("Key for INSERT or MERGE must not be null", IgniteQueryErrorCode.NULL_KEY); + if (key == null) { + if (F.isEmpty(desc.keyFieldName())) + throw new IgniteSQLException("Key for INSERT or MERGE must not be null", IgniteQueryErrorCode.NULL_KEY); + else + throw new IgniteSQLException("Null value is not allowed for column '" + desc.keyFieldName() + "'", + IgniteQueryErrorCode.NULL_KEY); + } - if (val == null) - throw new IgniteSQLException("Value for INSERT or MERGE must not be null", IgniteQueryErrorCode.NULL_VALUE); + if (val == null) { + if (F.isEmpty(desc.valueFieldName())) + throw new IgniteSQLException("Value for INSERT, MERGE, or UPDATE must not be null", + IgniteQueryErrorCode.NULL_VALUE); + else + throw new IgniteSQLException("Null value is not allowed for column '" + desc.valueFieldName() + "'", + IgniteQueryErrorCode.NULL_VALUE); + } Map newColVals = new HashMap<>(); http://git-wip-us.apache.org/repos/asf/ignite/blob/79d94cf4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Schema.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Schema.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Schema.java index 3f39e6a..f5cf0f2 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Schema.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Schema.java @@ -17,9 +17,8 @@ package org.apache.ignite.internal.processors.query.h2; -import org.jsr166.ConcurrentHashMap8; - import java.util.Collection; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; /** @@ -30,10 +29,10 @@ public class H2Schema { private final String schemaName; /** */ - private final ConcurrentMap tbls = new ConcurrentHashMap8<>(); + private final ConcurrentMap tbls = new ConcurrentHashMap<>(); /** */ - private final ConcurrentMap typeToTbl = new ConcurrentHashMap8<>(); + private final ConcurrentMap typeToTbl = new ConcurrentHashMap<>(); /** * Constructor. @@ -70,8 +69,8 @@ public class H2Schema { * @param typeName Type name. * @return Table. */ - public H2TableDescriptor tableByTypeName(String typeName) { - return typeToTbl.get(typeName); + public H2TableDescriptor tableByTypeName(String cacheName, String typeName) { + return typeToTbl.get(new H2TypeKey(cacheName, typeName)); } /** @@ -81,7 +80,7 @@ public class H2Schema { if (tbls.putIfAbsent(tbl.tableName(), tbl) != null) throw new IllegalStateException("Table already registered: " + tbl.fullTableName()); - if (typeToTbl.putIfAbsent(tbl.typeName(), tbl) != null) + if (typeToTbl.putIfAbsent(new H2TypeKey(tbl.cache().name(), tbl.typeName()), tbl) != null) throw new IllegalStateException("Table already registered: " + tbl.fullTableName()); } @@ -91,7 +90,7 @@ public class H2Schema { public void remove(H2TableDescriptor tbl) { tbls.remove(tbl.tableName()); - typeToTbl.remove(tbl.typeName()); + typeToTbl.remove(new H2TypeKey(tbl.cache().name(), tbl.typeName())); } /** @@ -104,7 +103,7 @@ public class H2Schema { tbls.remove(tbl.tableName()); - typeToTbl.remove(tbl.typeName()); + typeToTbl.remove(new H2TypeKey(tbl.cache().name(), tbl.typeName())); } /** http://git-wip-us.apache.org/repos/asf/ignite/blob/79d94cf4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TypeKey.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TypeKey.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TypeKey.java new file mode 100644 index 0000000..d39918c --- /dev/null +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TypeKey.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.query.h2; + +import org.apache.ignite.internal.util.typedef.F; + +/** + * Key for types lookup. + */ +public class H2TypeKey { + /** Cache name. */ + private final String cacheName; + + /** Type name. */ + private final String typeName; + + /** + * Constructor. + * + * @param cacheName Cache name. + * @param typeName Type name. + */ + H2TypeKey(String cacheName, String typeName) { + this.cacheName = cacheName; + this.typeName = typeName; + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (this == o) + return true; + + if (o == null || getClass() != o.getClass()) + return false; + + H2TypeKey other = (H2TypeKey)o; + + return F.eq(typeName, other.typeName) && F.eq(cacheName, other.cacheName); + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + int res = cacheName.hashCode(); + + res = 31 * res + typeName.hashCode(); + + return res; + } +} http://git-wip-us.apache.org/repos/asf/ignite/blob/79d94cf4/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 fd7b9a8..22ed592 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 @@ -548,7 +548,7 @@ public class IgniteH2Indexing implements GridQueryIndexing { GridCacheVersion ver, long expirationTime, long link) throws IgniteCheckedException { - H2TableDescriptor tbl = tableDescriptor(schema(cacheName), type.name()); + H2TableDescriptor tbl = tableDescriptor(schema(cacheName), cacheName, type.name()); if (tbl == null) return; // Type was rejected. @@ -572,7 +572,7 @@ public class IgniteH2Indexing implements GridQueryIndexing { if (log.isDebugEnabled()) log.debug("Removing key from cache query index [locId=" + nodeId + ", key=" + key + ", val=" + val + ']'); - H2TableDescriptor tbl = tableDescriptor(schema(cacheName), type.name()); + H2TableDescriptor tbl = tableDescriptor(schema(cacheName), cacheName, type.name()); if (tbl == null) return; @@ -777,10 +777,11 @@ public class IgniteH2Indexing implements GridQueryIndexing { } } + /** {@inheritDoc} */ @SuppressWarnings("unchecked") - @Override public GridCloseableIterator> queryLocalText(String schemaName, String qry, - String typeName, IndexingQueryFilter filters) throws IgniteCheckedException { - H2TableDescriptor tbl = tableDescriptor(schemaName, typeName); + @Override public GridCloseableIterator> queryLocalText(String schemaName, + String cacheName, String qry, String typeName, IndexingQueryFilter filters) throws IgniteCheckedException { + H2TableDescriptor tbl = tableDescriptor(schemaName, cacheName, typeName); if (tbl != null && tbl.luceneIndex() != null) { GridRunningQueryInfo run = new GridRunningQueryInfo(qryIdGen.incrementAndGet(), qry, TEXT, schemaName, @@ -1098,7 +1099,7 @@ public class IgniteH2Indexing implements GridQueryIndexing { /** {@inheritDoc} */ @SuppressWarnings("unchecked") - @Override public QueryCursor> queryLocalSql(String schemaName, + @Override public QueryCursor> queryLocalSql(String schemaName, String cacheName, final SqlQuery qry, final IndexingQueryFilter filter, final boolean keepBinary) throws IgniteCheckedException { String type = qry.getType(); String sqlQry = qry.getSql(); @@ -1107,7 +1108,7 @@ public class IgniteH2Indexing implements GridQueryIndexing { GridQueryCancel cancel = new GridQueryCancel(); - final GridCloseableIterator> i = queryLocalSql(schemaName, sqlQry, alias, + final GridCloseableIterator> i = queryLocalSql(schemaName, cacheName, sqlQry, alias, F.asList(params), type, filter, cancel); return new QueryCursorImpl<>(new Iterable>() { @@ -1142,19 +1143,19 @@ public class IgniteH2Indexing implements GridQueryIndexing { * Executes regular query. * * @param schemaName Schema name. + * @param cacheName Cache name. * @param qry Query. * @param alias Table alias. * @param params Query parameters. * @param type Query return type. - * @param filter Cache name and key filter. - * @return Queried rows. + * @param filter Cache name and key filter. @return Queried rows. * @throws IgniteCheckedException If failed. */ @SuppressWarnings("unchecked") - public GridCloseableIterator> queryLocalSql(String schemaName, + public GridCloseableIterator> queryLocalSql(String schemaName, String cacheName, final String qry, String alias, @Nullable final Collection params, String type, final IndexingQueryFilter filter, GridQueryCancel cancel) throws IgniteCheckedException { - final H2TableDescriptor tbl = tableDescriptor(schemaName, type); + final H2TableDescriptor tbl = tableDescriptor(schemaName, cacheName, type); if (tbl == null) throw new IgniteSQLException("Failed to find SQL table for type: " + type, @@ -1216,11 +1217,11 @@ public class IgniteH2Indexing implements GridQueryIndexing { /** {@inheritDoc} */ @SuppressWarnings("unchecked") - @Override public QueryCursor> queryDistributedSql(String schemaName, SqlQuery qry, - boolean keepBinary, int mainCacheId) { + @Override public QueryCursor> queryDistributedSql(String schemaName, String cacheName, + SqlQuery qry, boolean keepBinary, int mainCacheId) { String type = qry.getType(); - H2TableDescriptor tblDesc = tableDescriptor(schemaName, type); + H2TableDescriptor tblDesc = tableDescriptor(schemaName, cacheName, type); if (tblDesc == null) throw new IgniteSQLException("Failed to find SQL table for type: " + type, @@ -1829,13 +1830,13 @@ public class IgniteH2Indexing implements GridQueryIndexing { * @param type Type name. * @return Descriptor. */ - @Nullable private H2TableDescriptor tableDescriptor(String schemaName, String type) { + @Nullable private H2TableDescriptor tableDescriptor(String schemaName, String cacheName, String type) { H2Schema schema = schemas.get(schemaName); if (schema == null) return null; - return schema.tableByTypeName(type); + return schema.tableByTypeName(cacheName, type); } /** {@inheritDoc} */ http://git-wip-us.apache.org/repos/asf/ignite/blob/79d94cf4/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 affd903..f39e587 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 @@ -62,6 +62,7 @@ import org.h2.table.Column; import org.h2.value.DataType; import static org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing.UPDATE_RESULT_META; +import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlQueryParser.PARAM_WRAP_VALUE; /** * DDL statements processor.

@@ -227,6 +228,10 @@ public class DdlStatementsProcessor { cmd.tableName()); } else { + if (QueryUtils.isSqlType(tbl.rowDescriptor().type().valueClass())) + throw new SchemaOperationException("Cannot add column(s) because table was created " + + "with " + PARAM_WRAP_VALUE + "=false option."); + List cols = new ArrayList<>(cmd.columns().length); boolean allFieldsNullable = true; @@ -373,11 +378,38 @@ public class DdlStatementsProcessor { if (!F.isEmpty(createTbl.valueTypeName())) valTypeName = createTbl.valueTypeName(); + assert createTbl.wrapKey() != null; + assert createTbl.wrapValue() != null; + + if (!createTbl.wrapKey()) { + GridSqlColumn pkCol = createTbl.columns().get(createTbl.primaryKeyColumns().iterator().next()); + + keyTypeName = DataType.getTypeClassName(pkCol.column().getType()); + + res.setKeyFieldName(pkCol.columnName()); + } + else + res.setKeyFields(createTbl.primaryKeyColumns()); + + if (!createTbl.wrapValue()) { + GridSqlColumn valCol = null; + + for (Map.Entry e : createTbl.columns().entrySet()) { + if (!createTbl.primaryKeyColumns().contains(e.getKey())) { + valCol = e.getValue(); + + break; + } + } + + valTypeName = DataType.getTypeClassName(valCol.column().getType()); + + res.setValueFieldName(valCol.columnName()); + } + res.setValueType(valTypeName); res.setKeyType(keyTypeName); - res.setKeyFields(createTbl.primaryKeyColumns()); - if (!F.isEmpty(notNullFields)) { QueryEntityEx res0 = new QueryEntityEx(res); http://git-wip-us.apache.org/repos/asf/ignite/blob/79d94cf4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlCreateTable.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlCreateTable.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlCreateTable.java index b73214f..de10826 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlCreateTable.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlCreateTable.java @@ -71,6 +71,12 @@ public class GridSqlCreateTable extends GridSqlStatement { /** Name of the column that represents affinity key. */ private String affinityKey; + /** Forcefully turn single column PK into an Object. */ + private Boolean wrapKey; + + /** Forcefully turn single column value into an Object. */ + private Boolean wrapVal; + /** Extra WITH-params. */ private List params; @@ -271,6 +277,34 @@ public class GridSqlCreateTable extends GridSqlStatement { } /** + * @return Forcefully turn single column PK into an Object. + */ + public Boolean wrapKey() { + return wrapKey; + } + + /** + * @param wrapKey Forcefully turn single column PK into an Object. + */ + public void wrapKey(boolean wrapKey) { + this.wrapKey = wrapKey; + } + + /** + * @return Forcefully turn single column value into an Object. + */ + public Boolean wrapValue() { + return wrapVal; + } + + /** + * @param wrapVal Forcefully turn single column value into an Object.. + */ + public void wrapValue(boolean wrapVal) { + this.wrapVal = wrapVal; + } + + /** * @return Extra WITH-params. */ public List params() { http://git-wip-us.apache.org/repos/asf/ignite/blob/79d94cf4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java index 3d7a1a0..bf72200 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java @@ -475,6 +475,12 @@ public class GridSqlQueryParser { private static final String PARAM_VAL_TYPE = "VALUE_TYPE"; /** */ + private static final String PARAM_WRAP_KEY = "WRAP_KEY"; + + /** */ + public static final String PARAM_WRAP_VALUE = "WRAP_VALUE"; + + /** */ private final IdentityHashMap h2ObjToGridObj = new IdentityHashMap<>(); /** */ @@ -1007,7 +1013,8 @@ public class GridSqlQueryParser { pkCols.add(gridCol.columnName()); } - int valColsNum = cols.size() - pkCols.size(); + int keyColsNum = pkCols.size(); + int valColsNum = cols.size() - keyColsNum; if (valColsNum == 0) throw new IgniteSQLException("Table must have at least one non PRIMARY KEY column.", @@ -1052,6 +1059,44 @@ public class GridSqlQueryParser { processExtraParam(e.getKey(), e.getValue(), res); } + // Process key wrapping. + Boolean wrapKey = res.wrapKey(); + + if (wrapKey != null && !wrapKey) { + if (keyColsNum > 1) { + throw new IgniteSQLException(PARAM_WRAP_KEY + " cannot be false when composite primary key exists.", + IgniteQueryErrorCode.PARSING); + } + + if (!F.isEmpty(res.keyTypeName())) { + throw new IgniteSQLException(PARAM_WRAP_KEY + " cannot be false when " + PARAM_KEY_TYPE + " is set.", + IgniteQueryErrorCode.PARSING); + } + } + + boolean wrapKey0 = (res.wrapKey() != null && res.wrapKey()) || !F.isEmpty(res.keyTypeName()) || keyColsNum > 1; + + res.wrapKey(wrapKey0); + + // Process value wrapping. + Boolean wrapVal = res.wrapValue(); + + if (wrapVal != null && !wrapVal) { + if (valColsNum > 1) { + throw new IgniteSQLException(PARAM_WRAP_VALUE + " cannot be false when multiple non-primary key " + + "columns exist.", IgniteQueryErrorCode.PARSING); + } + + if (!F.isEmpty(res.valueTypeName())) { + throw new IgniteSQLException(PARAM_WRAP_VALUE + " cannot be false when " + PARAM_VAL_TYPE + " is set.", + IgniteQueryErrorCode.PARSING); + } + + res.wrapValue(false); + } + else + res.wrapValue(true); // By default value is always wrapped to allow for ALTER TABLE ADD COLUMN commands. + if (!F.isEmpty(res.valueTypeName()) && F.eq(res.keyTypeName(), res.valueTypeName())) throw new IgniteSQLException("Key and value type names " + "should be different for CREATE TABLE: " + res.valueTypeName(), IgniteQueryErrorCode.PARSING); @@ -1336,6 +1381,17 @@ public class GridSqlQueryParser { break; + case PARAM_WRAP_KEY: { + res.wrapKey(F.isEmpty(val) || Boolean.parseBoolean(val)); + + break; + } + + case PARAM_WRAP_VALUE: + res.wrapValue(F.isEmpty(val) || Boolean.parseBoolean(val)); + + break; + default: throw new IgniteSQLException("Unsupported parameter: " + name, IgniteQueryErrorCode.PARSING); } http://git-wip-us.apache.org/repos/asf/ignite/blob/79d94cf4/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsAbstractConcurrentSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsAbstractConcurrentSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsAbstractConcurrentSelfTest.java index 969c985..3d4b2a3 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsAbstractConcurrentSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsAbstractConcurrentSelfTest.java @@ -82,7 +82,7 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo private static final String ATTR_FILTERED = "FILTERED"; /** SQL statement to create test table accompanied by template specification. */ - private static final String CREATE_SQL_WITH_TEMPLATE = CREATE_SQL + " WITH \"template=TPL\""; + private final String createSql; /** Latches to block certain index operations. */ private static final ConcurrentHashMap> BLOCKS = @@ -103,6 +103,7 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo DynamicColumnsAbstractConcurrentSelfTest(CacheMode cacheMode, CacheAtomicityMode atomicityMode) { this.cacheMode = cacheMode; this.atomicityMode = atomicityMode; + createSql = CREATE_SQL + " WITH \"template=TPL\""; } /** {@inheritDoc} */ @@ -158,7 +159,7 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo createSqlCache(cli); - run(cli, CREATE_SQL_WITH_TEMPLATE); + run(cli, createSql); // Test migration between normal servers. CountDownLatch idxLatch = blockIndexing(srv1Id); @@ -210,7 +211,7 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo createSqlCache(srv1); - run(srv1, CREATE_SQL_WITH_TEMPLATE); + run(srv1, createSql); CountDownLatch idxLatch = blockIndexing(srv1); @@ -253,7 +254,7 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo createSqlCache(srv1); - run(srv1, CREATE_SQL_WITH_TEMPLATE); + run(srv1, createSql); CountDownLatch idxLatch = blockIndexing(srv1); @@ -296,7 +297,7 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo createSqlCache(srv1); - run(srv1, CREATE_SQL_WITH_TEMPLATE); + run(srv1, createSql); // Start data change operations from several threads. final AtomicBoolean stopped = new AtomicBoolean(); @@ -309,7 +310,7 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo int key = ThreadLocalRandom.current().nextInt(0, LARGE_CACHE_SIZE); int val = ThreadLocalRandom.current().nextInt(); - IgniteCache cache = node.cache(CACHE_NAME); + IgniteCache cache = node.cache(CACHE_NAME); if (ThreadLocalRandom.current().nextBoolean()) cache.put(key(node, key), val(node, val)); @@ -342,15 +343,15 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo // Get expected values. Set expKeys = new HashSet<>(); - IgniteCache cache = srv1.cache(CACHE_NAME).withKeepBinary(); + IgniteCache cache = srv1.cache(CACHE_NAME).withKeepBinary(); for (int i = 0; i < LARGE_CACHE_SIZE; i++) { - BinaryObject key = key(srv1, i); + Object key = key(srv1, i); BinaryObject val = cache.get(key); if (val != null) { - int id = key.field("ID"); + int id = (Integer)key; assertEquals(i, id); @@ -363,17 +364,17 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo // Validate query result. for (Ignite node : Ignition.allGrids()) { - IgniteCache nodeCache = node.cache(CACHE_NAME).withKeepBinary(); + IgniteCache nodeCache = node.cache(CACHE_NAME).withKeepBinary(); SqlQuery qry = new SqlQuery(valTypeName, "from " + TBL_NAME + " where mod(id, 2) <> 0"); - List> res = nodeCache.query(qry).getAll(); + List> res = nodeCache.query(qry).getAll(); assertEquals("Cache size mismatch [exp=" + expKeys.size() + ", actual=" + res.size() + ']', expKeys.size(), res.size()); - for (Cache.Entry entry : res) { - int key = entry.getKey().field("ID"); + for (Cache.Entry entry : res) { + int key = (Integer)entry.getKey(); int v = entry.getValue().field("v"); String name = entry.getValue().field("NAME"); @@ -399,13 +400,11 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo /** * @param node Node. - * @param key Value for ID field. - * @return PERSON cache key. + * @param id Key. + * @return PERSON cache key (int or {@link BinaryObject}). */ - private BinaryObject key(Ignite node, int key) { - String keyTypeName = ((IgniteEx)node).context().query().types(CACHE_NAME).iterator().next().keyTypeName(); - - return node.binary().builder(keyTypeName).setField("ID", key).build(); + private Object key(Ignite node, int id) { + return id; } /** @@ -420,7 +419,7 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo createSqlCache(srv1); - run(srv1, CREATE_SQL_WITH_TEMPLATE); + run(srv1, createSql); awaitPartitionMapExchange(); @@ -481,7 +480,7 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo // Start cache and populate it with data. createSqlCache(cli); - run(cli, CREATE_SQL_WITH_TEMPLATE); + run(cli, createSql); put(cli, 0, LARGE_CACHE_SIZE); @@ -526,7 +525,7 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo createSqlCache(cli); - run(cli, CREATE_SQL_WITH_TEMPLATE); + run(cli, createSql); put(cli, 0, 5000); @@ -539,8 +538,8 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo while (!stopped.get()) { Ignite node = grid(ThreadLocalRandom.current().nextInt(1, 5)); - IgniteInternalFuture fut = addCols(node, QueryUtils.DFLT_SCHEMA, c("newCol" + dynColCnt.getAndIncrement(), - Integer.class.getName())); + IgniteInternalFuture fut = addCols(node, QueryUtils.DFLT_SCHEMA, c("newCol" + + dynColCnt.getAndIncrement(), Integer.class.getName())); try { fut.get(); @@ -641,7 +640,7 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo if (dynamicCache) { createSqlCache(cli); - run(cli, CREATE_SQL_WITH_TEMPLATE); + run(cli, createSql); } final String schemaName = dynamicCache ? QueryUtils.DFLT_SCHEMA : "idx"; @@ -677,7 +676,7 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo if (dynamicCache) { DynamicColumnsAbstractConcurrentSelfTest.this.run(srvNode, DROP_SQL); - DynamicColumnsAbstractConcurrentSelfTest.this.run(srvNode, CREATE_SQL_WITH_TEMPLATE); + DynamicColumnsAbstractConcurrentSelfTest.this.run(srvNode, createSql); } else { srvNode.destroyCache("idx"); @@ -724,7 +723,7 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo createSqlCache(cli); - run(cli, CREATE_SQL_WITH_TEMPLATE); + run(cli, createSql); final AtomicBoolean stopped = new AtomicBoolean(); @@ -783,8 +782,8 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo while (!stopped.get()) { Ignite node = grid(ThreadLocalRandom.current().nextInt(1, 5)); - IgniteInternalFuture fut = addCols(node, QueryUtils.DFLT_SCHEMA, c("newCol" + dynColCnt.getAndIncrement(), - Integer.class.getName())); + IgniteInternalFuture fut = addCols(node, QueryUtils.DFLT_SCHEMA, c("newCol" + + dynColCnt.getAndIncrement(), Integer.class.getName())); try { fut.get(); http://git-wip-us.apache.org/repos/asf/ignite/blob/79d94cf4/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsConcurrentAtomicPartitionedSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsConcurrentAtomicPartitionedSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsConcurrentAtomicPartitionedSelfTest.java index ca68903..3f65fca 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsConcurrentAtomicPartitionedSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsConcurrentAtomicPartitionedSelfTest.java @@ -21,7 +21,7 @@ import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.cache.CacheMode; /** - * Test to check concurrent operations on dynamic columns on ATOMIC PARTITIONED cache. + * Test to check concurrent operations on dynamic columns on ATOMIC PARTITIONED cache with flat key. */ public class DynamicColumnsConcurrentAtomicPartitionedSelfTest extends DynamicColumnsAbstractConcurrentSelfTest { /** http://git-wip-us.apache.org/repos/asf/ignite/blob/79d94cf4/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsConcurrentAtomicReplicatedSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsConcurrentAtomicReplicatedSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsConcurrentAtomicReplicatedSelfTest.java index 9a3a32c..289a01e 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsConcurrentAtomicReplicatedSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsConcurrentAtomicReplicatedSelfTest.java @@ -21,7 +21,7 @@ import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.cache.CacheMode; /** - * Test to check concurrent operations on dynamic columns on ATOMIC REPLICATED cache. + * Test to check concurrent operations on dynamic columns on ATOMIC REPLICATED cache with flat key. */ public class DynamicColumnsConcurrentAtomicReplicatedSelfTest extends DynamicColumnsAbstractConcurrentSelfTest { /** http://git-wip-us.apache.org/repos/asf/ignite/blob/79d94cf4/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsConcurrentTransactionalPartitionedSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsConcurrentTransactionalPartitionedSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsConcurrentTransactionalPartitionedSelfTest.java index f42a447..371313e 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsConcurrentTransactionalPartitionedSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsConcurrentTransactionalPartitionedSelfTest.java @@ -21,7 +21,7 @@ import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.cache.CacheMode; /** - * Test to check concurrent operations on dynamic columns on TRANSACTIONAL PARTITIONED cache. + * Test to check concurrent operations on dynamic columns on TRANSACTIONAL PARTITIONED cache with flat key. */ public class DynamicColumnsConcurrentTransactionalPartitionedSelfTest extends DynamicColumnsAbstractConcurrentSelfTest { /** http://git-wip-us.apache.org/repos/asf/ignite/blob/79d94cf4/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsConcurrentTransactionalReplicatedSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsConcurrentTransactionalReplicatedSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsConcurrentTransactionalReplicatedSelfTest.java index 2b53e42..f004b89 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsConcurrentTransactionalReplicatedSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsConcurrentTransactionalReplicatedSelfTest.java @@ -21,9 +21,10 @@ import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.cache.CacheMode; /** - * Test to check concurrent operations on dynamic columns on TRANSACTIONAL REPLICATED cache. + * Test to check concurrent operations on dynamic columns on TRANSACTIONAL REPLICATED cache with flat key. */ -public class DynamicColumnsConcurrentTransactionalReplicatedSelfTest extends DynamicColumnsAbstractConcurrentSelfTest { +public class DynamicColumnsConcurrentTransactionalReplicatedSelfTest + extends DynamicColumnsAbstractConcurrentSelfTest { /** * Constructor. */ http://git-wip-us.apache.org/repos/asf/ignite/blob/79d94cf4/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicColumnsAbstractBasicSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicColumnsAbstractBasicSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicColumnsAbstractBasicSelfTest.java index 5e64959..34be34d 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicColumnsAbstractBasicSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicColumnsAbstractBasicSelfTest.java @@ -302,6 +302,50 @@ public abstract class H2DynamicColumnsAbstractBasicSelfTest extends DynamicColum } /** + * Test that {@code ADD COLUMN} fails for non dynamic table that has flat value. + */ + @SuppressWarnings({"unchecked", "ThrowFromFinallyBlock"}) + public void testTestAlterTableOnFlatValueNonDynamicTable() { + CacheConfiguration c = + new CacheConfiguration("ints").setIndexedTypes(Integer.class, Integer.class) + .setSqlSchema(QueryUtils.DFLT_SCHEMA); + + try { + grid(nodeIndex()).getOrCreateCache(c); + + doTestAlterTableOnFlatValue("INTEGER"); + } + finally { + grid(nodeIndex()).destroyCache("ints"); + } + } + + /** + * Test that {@code ADD COLUMN} fails for dynamic table that has flat value. + */ + @SuppressWarnings({"unchecked", "ThrowFromFinallyBlock"}) + public void testTestAlterTableOnFlatValueDynamicTable() { + try { + run("CREATE TABLE TEST (id int primary key, x varchar) with \"wrap_value=false\""); + + doTestAlterTableOnFlatValue("TEST"); + } + finally { + run("DROP TABLE TEST"); + } + } + + /** + * Test that {@code ADD COLUMN} fails for tables that have flat value. + * @param tblName table name. + */ + private void doTestAlterTableOnFlatValue(String tblName) { + assertThrows("ALTER TABLE " + tblName + " ADD COLUMN y varchar", + "ADD COLUMN is not supported for tables created with wrap_value=false param. " + + "(To enable ADD COLUMN, create table with wrap_value=true param)."); + } + + /** * @return Node index to run queries on. */ protected abstract int nodeIndex(); http://git-wip-us.apache.org/repos/asf/ignite/blob/79d94cf4/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicTableSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicTableSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicTableSelfTest.java index 773e7e0..e0ab6c5 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicTableSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicTableSelfTest.java @@ -17,6 +17,11 @@ package org.apache.ignite.internal.processors.cache.index; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -59,6 +64,7 @@ import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.testframework.GridTestUtils; import org.h2.jdbc.JdbcSQLException; +import org.h2.value.DataType; /** * Tests for CREATE/DROP TABLE. @@ -285,6 +291,14 @@ public class H2DynamicTableSelfTest extends AbstractSchemaSelfTest { } /** + * Test that {@code CREATE TABLE} with given write sync mode actually creates new cache as needed. + * @throws Exception if failed. + */ + public void testPlainKey() throws Exception { + doTestCreateTable(null, null, null, CacheWriteSynchronizationMode.FULL_SYNC); + } + + /** * Test that appending supplied arguments to {@code CREATE TABLE} results in creating new cache that has settings * as expected * @param cacheName Cache name, or {@code null} if the name generated by default should be used. @@ -292,18 +306,19 @@ public class H2DynamicTableSelfTest extends AbstractSchemaSelfTest { * @param valTypeName Value type name, or {@code null} if the name generated by default should be used. */ private void doTestCustomNames(String cacheName, String keyTypeName, String valTypeName) { - GridStringBuilder b = new GridStringBuilder("CREATE TABLE \"NameTest\" (id int primary key, x varchar) WITH "); + GridStringBuilder b = new GridStringBuilder("CREATE TABLE \"NameTest\" (id int primary key, x varchar) WITH " + + "wrap_key,wrap_value"); assert !F.isEmpty(cacheName) || !F.isEmpty(keyTypeName) || !F.isEmpty(valTypeName); if (!F.isEmpty(cacheName)) - b.a("\"cache_name=").a(cacheName).a('"').a(','); + b.a(",\"cache_name=").a(cacheName).a('"'); if (!F.isEmpty(keyTypeName)) - b.a("\"key_type=").a(keyTypeName).a('"').a(','); + b.a(",\"key_type=").a(keyTypeName).a('"'); if (!F.isEmpty(valTypeName)) - b.a("\"value_type=").a(valTypeName).a('"'); + b.a(",\"value_type=").a(valTypeName).a('"'); String res = b.toString(); @@ -312,8 +327,7 @@ public class H2DynamicTableSelfTest extends AbstractSchemaSelfTest { execute(client(), res); - String resCacheName = U.firstNotNull(cacheName, QueryUtils.createTableCacheName(QueryUtils.DFLT_SCHEMA, - "NameTest")); + String resCacheName = U.firstNotNull(cacheName, cacheName("NameTest")); IgniteInternalCache cache = client().cachex(resCacheName); @@ -770,7 +784,8 @@ public class H2DynamicTableSelfTest extends AbstractSchemaSelfTest { * @throws Exception if failed. */ public void testAffinityKey() throws Exception { - execute("CREATE TABLE \"City\" (\"name\" varchar primary key, \"code\" int) WITH \"affinityKey='name'\""); + execute("CREATE TABLE \"City\" (\"name\" varchar primary key, \"code\" int) WITH wrap_key,wrap_value," + + "\"affinityKey='name'\""); assertAffinityCacheConfiguration("City", "name"); @@ -783,7 +798,7 @@ public class H2DynamicTableSelfTest extends AbstractSchemaSelfTest { // We need unique name for this table to avoid conflicts with existing binary metadata. execute("CREATE TABLE \"Person2\" (\"id\" int, \"city\" varchar," + " \"name\" varchar, \"surname\" varchar, \"age\" int, PRIMARY KEY (\"id\", \"city\")) WITH " + - "\"template=cache,affinityKey='city'\""); + "wrap_key,wrap_value,\"template=cache,affinityKey='city'\""); assertAffinityCacheConfiguration("Person2", "city"); @@ -828,26 +843,31 @@ public class H2DynamicTableSelfTest extends AbstractSchemaSelfTest { */ @SuppressWarnings("ThrowableResultOfMethodCallIgnored") public void testAffinityKeyCaseSensitivity() { - execute("CREATE TABLE \"A\" (\"name\" varchar primary key, \"code\" int) WITH \"affinityKey='name'\""); + execute("CREATE TABLE \"A\" (\"name\" varchar primary key, \"code\" int) WITH wrap_key,wrap_value," + + "\"affinityKey='name'\""); assertAffinityCacheConfiguration("A", "name"); - execute("CREATE TABLE \"B\" (name varchar primary key, \"code\" int) WITH \"affinityKey=name\""); + execute("CREATE TABLE \"B\" (name varchar primary key, \"code\" int) WITH wrap_key,wrap_value," + + "\"affinityKey=name\""); assertAffinityCacheConfiguration("B", "NAME"); - execute("CREATE TABLE \"C\" (name varchar primary key, \"code\" int) WITH \"affinityKey=NamE\""); + execute("CREATE TABLE \"C\" (name varchar primary key, \"code\" int) WITH wrap_key,wrap_value," + + "\"affinityKey=NamE\""); assertAffinityCacheConfiguration("C", "NAME"); - execute("CREATE TABLE \"D\" (\"name\" varchar primary key, \"code\" int) WITH \"affinityKey=NAME\""); + execute("CREATE TABLE \"D\" (\"name\" varchar primary key, \"code\" int) WITH wrap_key,wrap_value," + + "\"affinityKey=NAME\""); assertAffinityCacheConfiguration("D", "name"); // Error arises because user has specified case sensitive affinity column name GridTestUtils.assertThrows(null, new Callable() { @Override public Object call() throws Exception { - execute("CREATE TABLE \"E\" (name varchar primary key, \"code\" int) WITH \"affinityKey='Name'\""); + execute("CREATE TABLE \"E\" (name varchar primary key, \"code\" int) WITH wrap_key,wrap_value," + + "\"affinityKey='Name'\""); return null; } @@ -865,9 +885,19 @@ public class H2DynamicTableSelfTest extends AbstractSchemaSelfTest { }, IgniteSQLException.class, "Ambiguous affinity column name, use single quotes for case sensitivity: name"); execute("CREATE TABLE \"E\" (\"name\" varchar, \"Name\" int, val int, primary key(\"name\", " + - "\"Name\")) WITH \"affinityKey='Name'\""); + "\"Name\")) WITH wrap_key,wrap_value,\"affinityKey='Name'\""); assertAffinityCacheConfiguration("E", "Name"); + + execute("drop table a"); + + execute("drop table b"); + + execute("drop table c"); + + execute("drop table d"); + + execute("drop table e"); } /** @@ -914,7 +944,7 @@ public class H2DynamicTableSelfTest extends AbstractSchemaSelfTest { execute("create index on \"PUBLIC\".t (b desc)"); execute("drop table \"PUBLIC\".t"); - assertNull(client().cache("t")); + assertNull(client().cache(cacheName("t"))); execute("create table \"PUBLIC\".t (a int primary key, b varchar(30))"); @@ -944,6 +974,224 @@ public class H2DynamicTableSelfTest extends AbstractSchemaSelfTest { } /** + * Test that it's impossible to create tables with same name regardless of key/value wrapping settings. + */ + public void testWrappedAndUnwrappedKeyTablesInteroperability() { + { + execute("create table a (id int primary key, x varchar)"); + + assertDdlCommandThrows("create table a (id int primary key, x varchar) with wrap_key", + "Table already exists: A"); + + assertDdlCommandThrows("create table a (id int primary key, x varchar) with wrap_value", + "Table already exists: A"); + + assertDdlCommandThrows("create table a (id int primary key, x varchar) with wrap_key,wrap_value", + "Table already exists: A"); + + execute("drop table a"); + } + + { + execute("create table a (id int primary key, x varchar) with wrap_key"); + + assertDdlCommandThrows("create table a (id int primary key, x varchar)", + "Table already exists: A"); + + assertDdlCommandThrows("create table a (id int primary key, x varchar) with wrap_value", + "Table already exists: A"); + + assertDdlCommandThrows("create table a (id int primary key, x varchar) with wrap_key,wrap_value", + "Table already exists: A"); + + execute("drop table a"); + } + + { + execute("create table a (id int primary key, x varchar) with wrap_value"); + + assertDdlCommandThrows("create table a (id int primary key, x varchar)", + "Table already exists: A"); + + assertDdlCommandThrows("create table a (id int primary key, x varchar) with wrap_key", + "Table already exists: A"); + + assertDdlCommandThrows("create table a (id int primary key, x varchar) with wrap_key,wrap_value", + "Table already exists: A"); + + execute("drop table a"); + } + + { + execute("create table a (id int primary key, x varchar) with wrap_key,wrap_value"); + + assertDdlCommandThrows("create table a (id int primary key, x varchar)", + "Table already exists: A"); + + assertDdlCommandThrows("create table a (id int primary key, x varchar) with wrap_value", + "Table already exists: A"); + + assertDdlCommandThrows("create table a (id int primary key, x varchar) with wrap_key", + "Table already exists: A"); + + execute("drop table a"); + } + } + + /** + * Test that it's possible to create tables with matching key and/or value primitive types. + */ + public void testDynamicTablesInteroperability() { + execute("create table a (id int primary key, x varchar) with \"wrap_value=false\""); + + execute("create table b (id long primary key, y varchar) with \"wrap_value=false\""); + + execute("create table c (id int primary key, z long) with \"wrap_value=false\""); + + execute("create table d (id int primary key, w varchar) with \"wrap_value=false\""); + + execute("drop table a"); + + execute("drop table b"); + + execute("drop table c"); + + execute("drop table d"); + } + + /** + * Test that when key or value has more than one column, wrap=false is forbidden. + */ + public void testWrappingAlwaysOnWithComplexObjects() { + assertDdlCommandThrows("create table a (id int, x varchar, c long, primary key(id, c)) with \"wrap_key=false\"", + "WRAP_KEY cannot be false when composite primary key exists."); + + assertDdlCommandThrows("create table a (id int, x varchar, c long, primary key(id)) with \"wrap_value=false\"", + "WRAP_VALUE cannot be false when multiple non-primary key columns exist."); + } + + /** + * Test behavior when neither key nor value should be wrapped. + * @throws SQLException if failed. + */ + public void testNoWrap() throws SQLException { + doTestKeyValueWrap(false, false); + } + + /** + * Test behavior when only key is wrapped. + * @throws SQLException if failed. + */ + public void testKeyWrap() throws SQLException { + doTestKeyValueWrap(true, false); + } + + /** + * Test behavior when only value is wrapped. + * @throws SQLException if failed. + */ + public void testValueWrap() throws SQLException { + doTestKeyValueWrap(false, true); + } + + /** + * Test behavior when both key and value is wrapped. + * @throws SQLException if failed. + */ + public void testKeyAndValueWrap() throws SQLException { + doTestKeyValueWrap(true, true); + } + + /** + * Test behavior for given combination of wrap flags. + * @param wrapKey Whether key wrap should be enforced. + * @param wrapVal Whether value wrap should be enforced. + * @throws SQLException if failed. + */ + private void doTestKeyValueWrap(boolean wrapKey, boolean wrapVal) throws SQLException { + try { + String sql = String.format("CREATE TABLE T (\"id\" int primary key, \"x\" varchar) WITH " + + "\"wrap_key=%b,wrap_value=%b\"", wrapKey, wrapVal); + + if (wrapKey) + sql += ",\"key_type=tkey\""; + + if (wrapVal) + sql += ",\"value_type=tval\""; + + execute(sql); + + execute("INSERT INTO T(\"id\", \"x\") values(1, 'a')"); + + LinkedHashMap resCols = new LinkedHashMap<>(); + + List resData = new ArrayList<>(); + + try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1")) { + try (ResultSet colsRs = conn.getMetaData().getColumns(null, QueryUtils.DFLT_SCHEMA, "T", ".*")) { + while (colsRs.next()) + resCols.put(colsRs.getString("COLUMN_NAME"), + DataType.getTypeClassName(DataType.convertSQLTypeToValueType(colsRs + .getShort("DATA_TYPE")))); + } + + try (PreparedStatement ps = conn.prepareStatement("SELECT * FROM T")) { + try (ResultSet dataRs = ps.executeQuery()) { + assertTrue(dataRs.next()); + + for (int i = 0; i < dataRs.getMetaData().getColumnCount(); i++) + resData.add(dataRs.getObject(i + 1)); + } + } + } + + LinkedHashMap expCols = new LinkedHashMap<>(); + + expCols.put("id", Integer.class.getName()); + expCols.put("x", String.class.getName()); + + assertEquals(expCols, resCols); + + assertEqualsCollections(Arrays.asList(1, "a"), resData); + + Object key = createKeyForWrapTest(1, wrapKey); + + Object val = client().cache(cacheName("T")).withKeepBinary().get(key); + + assertNotNull(val); + + assertEquals(createValueForWrapTest("a", wrapVal), val); + } + finally { + execute("DROP TABLE IF EXISTS T"); + } + } + + /** + * @param key Key to wrap. + * @param wrap Whether key should be wrapped. + * @return (optionally wrapped) key. + */ + private Object createKeyForWrapTest(int key, boolean wrap) { + if (!wrap) + return key; + + return client().binary().builder("tkey").setField("id", key).build(); + } + + /** + * @param val Value to wrap. + * @param wrap Whether value should be wrapped. + * @return (optionally wrapped) value. + */ + private Object createValueForWrapTest(String val, boolean wrap) { + if (!wrap) + return val; + + return client().binary().builder("tval").setField("x", val).build(); + } + + /** * Fill re-created table with data. */ private void fillRecreatedTable() { @@ -1025,19 +1273,30 @@ public class H2DynamicTableSelfTest extends AbstractSchemaSelfTest { } /** - * Test that {@code DROP TABLE} on non-public schema causes an exception. - * - * @throws Exception if failed. + * Test that arbitrary command yields specific error. + * @param cmd Command. + * @param expErrMsg Expected error message. */ @SuppressWarnings("ThrowableResultOfMethodCallIgnored") - public void testDropTableNotPublicSchema() throws Exception { + private void assertDdlCommandThrows(final String cmd, String expErrMsg) { GridTestUtils.assertThrows(null, new Callable() { @Override public Object call() throws Exception { - execute("DROP TABLE \"cache_idx\".\"Person\""); + execute(cmd); return null; } - }, IgniteSQLException.class, "DROP TABLE can only be executed on PUBLIC schema."); + }, IgniteSQLException.class, expErrMsg); + } + + /** + * Test that {@code DROP TABLE} on non-public schema causes an exception. + * + * @throws Exception if failed. + */ + @SuppressWarnings("ThrowableResultOfMethodCallIgnored") + public void testDropTableNotPublicSchema() throws Exception { + assertDdlCommandThrows("DROP TABLE \"cache_idx\".\"Person\"", + "DROP TABLE can only be executed on PUBLIC schema."); } /** http://git-wip-us.apache.org/repos/asf/ignite/blob/79d94cf4/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlNotNullConstraintTest.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlNotNullConstraintTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlNotNullConstraintTest.java index b372eb3..8deb61f 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlNotNullConstraintTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlNotNullConstraintTest.java @@ -86,7 +86,7 @@ public class IgniteSqlNotNullConstraintTest extends GridCommonAbstractTest { private static String CACHE_INTERCEPTOR = "cacheInterceptor"; /** Expected error message. */ - private static String ERR_MSG = "Null value is not allowed for field 'NAME'"; + private static String ERR_MSG = "Null value is not allowed for column 'NAME'"; /** Expected error message for read-through restriction. */ private static String READ_THROUGH_ERR_MSG = "NOT NULL constraint is not supported when " + http://git-wip-us.apache.org/repos/asf/ignite/blob/79d94cf4/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java index 1ee8a84..62860c0 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 @@ -251,20 +251,21 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract IgniteCache cacheB = ignite0.createCache(cacheBCfg()); - assertFalse(spi.queryLocalSql(spi.schema(typeAA.cacheName()), "select * from A.A", null, Collections.emptySet(), - typeAA.name(), null, null).hasNext()); + assertFalse(spi.queryLocalSql(spi.schema(typeAA.cacheName()), typeAA.cacheName(), "select * from A.A", null, + Collections.emptySet(), typeAA.name(), null, null).hasNext()); - assertFalse(spi.queryLocalSql(spi.schema(typeAB.cacheName()), "select * from A.B", null, Collections.emptySet(), - typeAB.name(), null, null).hasNext()); - - assertFalse(spi.queryLocalSql(spi.schema(typeBA.cacheName()), "select * from B.A", null, Collections.emptySet(), - typeBA.name(), null, null).hasNext()); + assertFalse(spi.queryLocalSql(spi.schema(typeAB.cacheName()), typeAB.cacheName(), "select * from A.B", null, + Collections.emptySet(), typeAB.name(), null, null).hasNext()); - assertFalse(spi.queryLocalSql(spi.schema(typeBA.cacheName()), "select * from B.A, A.B, A.A", null, + assertFalse(spi.queryLocalSql(spi.schema(typeBA.cacheName()), typeBA.cacheName(), "select * from B.A", null, Collections.emptySet(), typeBA.name(), null, null).hasNext()); + assertFalse(spi.queryLocalSql(spi.schema(typeBA.cacheName()), typeBA.cacheName(), + "select * from B.A, A.B, A.A", null, Collections.emptySet(), typeBA.name(), null, null).hasNext()); + try { - spi.queryLocalSql(spi.schema(typeBA.cacheName()), "select aa.*, ab.*, ba.* from A.A aa, A.B ab, B.A ba", + spi.queryLocalSql(spi.schema(typeBA.cacheName()), 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"); @@ -273,11 +274,11 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract // all fine } - assertFalse(spi.queryLocalSql(spi.schema(typeAB.cacheName()), "select ab.* from A.B ab", null, - Collections.emptySet(), typeAB.name(), null, null).hasNext()); + assertFalse(spi.queryLocalSql(spi.schema(typeAB.cacheName()), typeAB.cacheName(), "select ab.* from A.B ab", + null, Collections.emptySet(), typeAB.name(), null, null).hasNext()); - assertFalse(spi.queryLocalSql(spi.schema(typeBA.cacheName()), "select ba.* from B.A as ba", null, - Collections.emptySet(), typeBA.name(), null, null).hasNext()); + assertFalse(spi.queryLocalSql(spi.schema(typeBA.cacheName()), 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()); cacheA.put(1, ab(1, "Vasya", 20, "Some text about Vasya goes here.").build()); @@ -289,7 +290,7 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract // Query data. Iterator> res = spi.queryLocalSql(spi.schema(typeAA.cacheName()), - "from a order by age", null, Collections.emptySet(), typeAA.name(), null, null); + 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())); @@ -297,8 +298,8 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract assertEquals(aa("A", 2, "Valera", 19).build(), value(res.next())); assertFalse(res.hasNext()); - res = spi.queryLocalSql(spi.schema(typeAA.cacheName()), "select aa.* from a aa order by aa.age", null, - Collections.emptySet(), typeAA.name(), null, null); + res = spi.queryLocalSql(spi.schema(typeAA.cacheName()), typeAA.cacheName(), + "select aa.* from a aa order by aa.age", null, Collections.emptySet(), typeAA.name(), null, null); assertTrue(res.hasNext()); assertEquals(aa("A", 3, "Borya", 18).build(), value(res.next())); @@ -306,8 +307,8 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract assertEquals(aa("A", 2, "Valera", 19).build(), value(res.next())); assertFalse(res.hasNext()); - res = spi.queryLocalSql(spi.schema(typeAB.cacheName()), "from b order by name", null, Collections.emptySet(), - typeAB.name(), null, null); + res = spi.queryLocalSql(spi.schema(typeAB.cacheName()), 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())); @@ -315,8 +316,8 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract assertEquals(ab(4, "Vitalya", 20, "Very Good guy").build(), value(res.next())); assertFalse(res.hasNext()); - res = spi.queryLocalSql(spi.schema(typeAB.cacheName()), "select bb.* from b as bb order by bb.name", null, - Collections.emptySet(), typeAB.name(), null, null); + res = spi.queryLocalSql(spi.schema(typeAB.cacheName()), typeAB.cacheName(), + "select bb.* from b as bb order by bb.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())); @@ -324,8 +325,8 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract assertEquals(ab(4, "Vitalya", 20, "Very Good guy").build(), value(res.next())); assertFalse(res.hasNext()); - res = spi.queryLocalSql(spi.schema(typeBA.cacheName()), "from a", null, Collections.emptySet(), typeBA.name(), - null, null); + res = spi.queryLocalSql(spi.schema(typeBA.cacheName()), typeBA.cacheName(), "from a", null, + Collections.emptySet(), typeBA.name(), null, null); assertTrue(res.hasNext()); assertEquals(ba(2, "Kolya", 25, true).build(), value(res.next())); @@ -333,7 +334,7 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract // Text queries Iterator> txtRes = spi.queryLocalText(spi.schema(typeAB.cacheName()), - "good", typeAB.name(), null); + typeAB.cacheName(), "good", typeAB.name(), null); assertTrue(txtRes.hasNext()); assertEquals(ab(4, "Vitalya", 20, "Very Good guy").build(), value(txtRes.next())); http://git-wip-us.apache.org/repos/asf/ignite/blob/79d94cf4/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheDmlQueriesTest.cs ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheDmlQueriesTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheDmlQueriesTest.cs index a6ddc8c..172cb90 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheDmlQueriesTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheDmlQueriesTest.cs @@ -122,7 +122,7 @@ namespace Apache.Ignite.Core.Tests.Cache.Query var ex = Assert.Throws(() => cache.QueryFields(new SqlFieldsQuery( "insert into foo(_key, name) values (?, ?)", 1, "bar")).GetAll()); - Assert.AreEqual("Null value is not allowed for field 'ID'", ex.Message); + Assert.AreEqual("Null value is not allowed for column 'ID'", ex.Message); } /// @@ -137,7 +137,7 @@ namespace Apache.Ignite.Core.Tests.Cache.Query var ex = Assert.Throws(() => cache.QueryFields(new SqlFieldsQuery( "insert into foo(_key, id) values (?, ?)", 1, 2)).GetAll()); - Assert.AreEqual("Null value is not allowed for field 'NAME'", ex.Message); + Assert.AreEqual("Null value is not allowed for column 'NAME'", ex.Message); } ///