You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ag...@apache.org on 2017/04/26 09:31:23 UTC

[02/29] ignite git commit: IGNITE-5058: Fixed QueryIndex validation when QueryEntity.valueType is not set yet. This closes #1861.

IGNITE-5058: Fixed QueryIndex validation when QueryEntity.valueType is not set yet. This closes #1861.


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

Branch: refs/heads/ignite-5024
Commit: 1968e4f59eea6265aaf82a3fa515eecc99a91310
Parents: 09b6507
Author: devozerov <vo...@gridgain.com>
Authored: Mon Apr 24 12:31:51 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Mon Apr 24 12:31:51 2017 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/cache/QueryEntity.java    |  43 ++---
 .../processors/cache/GridCacheProcessor.java    |   4 +
 .../internal/processors/query/QuerySchema.java  |   4 +-
 .../internal/processors/query/QueryUtils.java   |  79 ++++++++-
 .../query/h2/H2IndexingAbstractGeoSelfTest.java |   2 +-
 .../DynamicIndexAbstractBasicSelfTest.java      |  16 +-
 .../index/QueryEntityValidationSelfTest.java    | 162 +++++++++++++++++++
 .../IgniteCacheQuerySelfTestSuite.java          |   3 +
 8 files changed, 278 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/1968e4f5/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java b/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java
index 31fe264..806cd7d 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java
@@ -18,16 +18,18 @@
 package org.apache.ignite.cache;
 
 import java.io.Serializable;
+import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.LinkedHashMap;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.ignite.internal.processors.query.QueryUtils;
-import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.internal.A;
+import org.apache.ignite.internal.util.typedef.internal.S;
 
 /**
  * Query entity is a description of {@link org.apache.ignite.IgniteCache cache} entry (composed of key and value)
@@ -44,16 +46,20 @@ public class QueryEntity implements Serializable {
     private String valType;
 
     /** Fields available for query. A map from field name to type name. */
+    @GridToStringInclude
     private LinkedHashMap<String, String> fields = new LinkedHashMap<>();
 
     /** Set of field names that belong to the key. */
+    @GridToStringInclude
     private Set<String> keyFields;
 
     /** Aliases. */
+    @GridToStringInclude
     private Map<String, String> aliases = new HashMap<>();
 
     /** Collection of query indexes. */
-    private Map<String, QueryIndex> idxs = new HashMap<>();
+    @GridToStringInclude
+    private Collection<QueryIndex> idxs;
 
     /** Table name. */
     private String tableName;
@@ -78,7 +84,7 @@ public class QueryEntity implements Serializable {
         keyFields = other.keyFields != null ? new HashSet<>(other.keyFields) : null;
 
         aliases = new HashMap<>(other.aliases);
-        idxs = new HashMap<>(other.idxs);
+        idxs = other.idxs != null ? new ArrayList<>(other.idxs) : null;
 
         tableName = other.tableName;
     }
@@ -190,7 +196,7 @@ public class QueryEntity implements Serializable {
      * @return Collection of index entities.
      */
     public Collection<QueryIndex> getIndexes() {
-        return idxs.values();
+        return idxs == null ? Collections.<QueryIndex>emptyList() : idxs;
     }
 
     /**
@@ -222,32 +228,12 @@ public class QueryEntity implements Serializable {
      * @return {@code this} for chaining.
      */
     public QueryEntity setIndexes(Collection<QueryIndex> idxs) {
-        for (QueryIndex idx : idxs) {
-            if (!F.isEmpty(idx.getFields())) {
-                if (idx.getName() == null)
-                    idx.setName(QueryUtils.indexName(this, idx));
-
-                if (idx.getIndexType() == null)
-                    throw new IllegalArgumentException("Index type is not set " + idx.getName());
-
-                if (!this.idxs.containsKey(idx.getName()))
-                    this.idxs.put(idx.getName(), idx);
-                else
-                    throw new IllegalArgumentException("Duplicate index name: " + idx.getName());
-            }
-        }
+        this.idxs = idxs;
 
         return this;
     }
 
     /**
-     * Clear indexes.
-     */
-    public void clearIndexes() {
-        this.idxs.clear();
-    }
-
-    /**
      * Gets table name for this query entity.
      *
      * @return table name
@@ -282,4 +268,9 @@ public class QueryEntity implements Serializable {
 
         return this;
     }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(QueryEntity.class, this);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/1968e4f5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index 28ef22f..a555b55 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -512,6 +512,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         if (cc.getEvictionPolicy() != null && !cc.isOnheapCacheEnabled())
             throw new IgniteCheckedException("Onheap cache must be enabled if eviction policy is configured [cacheName="
                 + U.maskName(cc.getName()) + "]");
+
+        QueryUtils.validateCacheConfiguration(cc);
     }
 
     /**
@@ -1461,6 +1463,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         CacheStore cfgStore = cfg.getCacheStoreFactory() != null ? cfg.getCacheStoreFactory().create() : null;
 
+        QueryUtils.prepareCacheConfiguration(cfg);
+
         validate(ctx.config(), cfg, cacheType, cfgStore);
 
         if (pluginMgr == null)

http://git-wip-us.apache.org/repos/asf/ignite/blob/1968e4f5/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QuerySchema.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QuerySchema.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QuerySchema.java
index 395f077..b380131 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QuerySchema.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QuerySchema.java
@@ -111,7 +111,6 @@ public class QuerySchema implements Serializable {
 
                             idxs.add(op0.index());
 
-                            entity.clearIndexes();
                             entity.setIndexes(idxs);
                         }
 
@@ -142,8 +141,7 @@ public class QuerySchema implements Serializable {
 
                         newIdxs.remove(victim);
 
-                        entity.clearIndexes();
-                        entity.setIndexes(idxs);
+                        entity.setIndexes(newIdxs);
 
                         break;
                     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/1968e4f5/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
index 3a7437b..e56f39f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
@@ -45,6 +45,7 @@ import java.lang.reflect.Method;
 import java.math.BigDecimal;
 import java.sql.Time;
 import java.sql.Timestamp;
+import java.util.Collection;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
@@ -162,9 +163,6 @@ public class QueryUtils {
      */
     public static QueryTypeCandidate typeForQueryEntity(String space, GridCacheContext cctx, QueryEntity qryEntity,
         List<Class<?>> mustDeserializeClss) throws IgniteCheckedException {
-        if (F.isEmpty(qryEntity.getValueType()))
-            throw new IgniteCheckedException("Value type is not set: " + qryEntity);
-
         GridKernalContext ctx = cctx.kernalContext();
         CacheConfiguration<?,?> ccfg = cctx.config();
 
@@ -797,6 +795,81 @@ public class QueryUtils {
     }
 
     /**
+     * Prepare cache configuration.
+     *
+     * @param ccfg Cache configuration.
+     */
+    @SuppressWarnings("unchecked")
+    public static void prepareCacheConfiguration(CacheConfiguration ccfg) {
+        assert ccfg != null;
+
+        Collection<QueryEntity> entities = ccfg.getQueryEntities();
+
+        if (!F.isEmpty(entities)) {
+            for (QueryEntity entity : entities) {
+                if (F.isEmpty(entity.getValueType()))
+                    continue;
+
+                Collection<QueryIndex> idxs = entity.getIndexes();
+
+                if (!F.isEmpty(idxs)) {
+                    for (QueryIndex idx : idxs) {
+                        if (idx.getName() == null) {
+                            String idxName = indexName(entity, idx);
+
+                            idx.setName(idxName);
+                        }
+                    }
+                }
+            }
+        }
+    }
+
+    /**
+     * Prepare cache configuration.
+     *
+     * @param ccfg Cache configuration.
+     * @throws IgniteCheckedException If failed.
+     */
+    @SuppressWarnings("unchecked")
+    public static void validateCacheConfiguration(CacheConfiguration ccfg) throws IgniteCheckedException {
+        assert ccfg != null;
+
+        Collection<QueryEntity> entities = ccfg.getQueryEntities();
+
+        if (!F.isEmpty(entities)) {
+            for (QueryEntity entity : entities) {
+                if (F.isEmpty(entity.getValueType()))
+                    throw new IgniteCheckedException("Value type cannot be null or empty [cacheName=" +
+                        ccfg.getName() + ", queryEntity=" + entity + ']');
+
+                Collection<QueryIndex> idxs = entity.getIndexes();
+
+                if (!F.isEmpty(idxs)) {
+                    Set<String> idxNames = new HashSet<>();
+
+                    for (QueryIndex idx : idxs) {
+                        String idxName = idx.getName();
+
+                        if (idxName == null)
+                            idxName = indexName(entity, idx);
+
+                        assert !F.isEmpty(idxName);
+
+                        if (!idxNames.add(idxName))
+                            throw new IgniteCheckedException("Duplicate index name [cacheName=" + ccfg.getName() +
+                                ", queryEntity=" + entity + ", queryIdx=" + idx + ']');
+
+                        if (idx.getIndexType() == null)
+                            throw new IgniteCheckedException("Index type is not set [cacheName=" + ccfg.getName() +
+                                ", queryEntity=" + entity + ", queryIdx=" + idx + ']');
+                    }
+                }
+            }
+        }
+    }
+
+    /**
      * Private constructor.
      */
     private QueryUtils() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/1968e4f5/modules/geospatial/src/test/java/org/apache/ignite/internal/processors/query/h2/H2IndexingAbstractGeoSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/geospatial/src/test/java/org/apache/ignite/internal/processors/query/h2/H2IndexingAbstractGeoSelfTest.java b/modules/geospatial/src/test/java/org/apache/ignite/internal/processors/query/h2/H2IndexingAbstractGeoSelfTest.java
index 80e9f93..f57d573 100644
--- a/modules/geospatial/src/test/java/org/apache/ignite/internal/processors/query/h2/H2IndexingAbstractGeoSelfTest.java
+++ b/modules/geospatial/src/test/java/org/apache/ignite/internal/processors/query/h2/H2IndexingAbstractGeoSelfTest.java
@@ -151,7 +151,7 @@ public abstract class H2IndexingAbstractGeoSelfTest extends GridCacheAbstractSel
 
             Collection<QueryIndex> idxs = new ArrayList<>(entity.getIndexes());
 
-            entity.clearIndexes();
+            entity.setIndexes(null);
 
             IgniteCache<K, V> cache = grid(0).getOrCreateCache(ccfg);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/1968e4f5/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractBasicSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractBasicSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractBasicSelfTest.java
index fc3529b..6621bb4 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractBasicSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractBasicSelfTest.java
@@ -670,23 +670,35 @@ public abstract class DynamicIndexAbstractBasicSelfTest extends DynamicIndexAbst
     public void checkDrop(CacheMode mode, CacheAtomicityMode atomicityMode, boolean near) throws Exception {
         initialize(mode, atomicityMode, near);
 
-        QueryIndex idx = index(IDX_NAME_1, field(FIELD_NAME_1));
+        // Create target index.
+        QueryIndex idx1 = index(IDX_NAME_1, field(FIELD_NAME_1));
 
-        dynamicIndexCreate(CACHE_NAME, TBL_NAME, idx, false);
+        dynamicIndexCreate(CACHE_NAME, TBL_NAME, idx1, false);
         assertIndex(CACHE_NAME, TBL_NAME, IDX_NAME_1, field(FIELD_NAME_1));
 
         assertIndexUsed(IDX_NAME_1, SQL_SIMPLE_FIELD_1, SQL_ARG_1);
 
         assertSimpleIndexOperations(SQL_SIMPLE_FIELD_1);
 
+        // Create another index which must stay intact afterwards.
+        QueryIndex idx2 = index(IDX_NAME_2, field(alias(FIELD_NAME_2)));
+
+        dynamicIndexCreate(CACHE_NAME, TBL_NAME, idx2, false);
+        assertIndex(CACHE_NAME, TBL_NAME, IDX_NAME_2, field(alias(FIELD_NAME_2)));
+
+        // Load some data.
         loadInitialData();
 
+        // Drop index.
         dynamicIndexDrop(CACHE_NAME, IDX_NAME_1, false);
         assertNoIndex(CACHE_NAME, TBL_NAME, IDX_NAME_1);
 
         assertSimpleIndexOperations(SQL_SIMPLE_FIELD_1);
 
         assertIndexNotUsed(IDX_NAME_1, SQL_SIMPLE_FIELD_1, SQL_ARG_1);
+
+        // Make sure the second index is still there.
+        assertIndex(CACHE_NAME, TBL_NAME, IDX_NAME_2, field(alias(FIELD_NAME_2)));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/1968e4f5/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/QueryEntityValidationSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/QueryEntityValidationSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/QueryEntityValidationSelfTest.java
new file mode 100644
index 0000000..97c9aa1
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/QueryEntityValidationSelfTest.java
@@ -0,0 +1,162 @@
+/*
+ * 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.cache.index;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cache.QueryEntity;
+import org.apache.ignite.cache.QueryIndex;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.concurrent.Callable;
+
+/**
+ * Tests for query entity validation.
+ */
+@SuppressWarnings("ThrowableResultOfMethodCallIgnored")
+public class QueryEntityValidationSelfTest extends GridCommonAbstractTest {
+    /** Cache name. */
+    private static final String CACHE_NAME = "cache";
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGrid(0);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * Test null value type.
+     *
+     * @throws Exception If failed.
+     */
+    public void testValueTypeNull() throws Exception {
+        final CacheConfiguration ccfg = new CacheConfiguration().setName(CACHE_NAME);
+
+        QueryEntity entity = new QueryEntity();
+
+        entity.setKeyType("Key");
+
+        ccfg.setQueryEntities(Collections.singleton(entity));
+
+        GridTestUtils.assertThrows(log, new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                grid(0).createCache(ccfg);
+
+                return null;
+            }
+        }, IgniteCheckedException.class, "Value type cannot be null or empty");
+    }
+
+    /**
+     * Test failure if index type is null.
+     *
+     * @throws Exception If failed.
+     */
+    public void testIndexTypeNull() throws Exception {
+        final CacheConfiguration ccfg = new CacheConfiguration().setName(CACHE_NAME);
+
+        QueryEntity entity = new QueryEntity();
+
+        entity.setKeyType("Key");
+        entity.setValueType("Value");
+
+        LinkedHashMap<String, String> fields = new LinkedHashMap<>();
+
+        fields.put("a", Integer.class.getName());
+
+        entity.setFields(fields);
+
+        LinkedHashMap<String, Boolean> idxFields = new LinkedHashMap<>();
+
+        idxFields.put("a", true);
+
+        QueryIndex idx = new QueryIndex().setName("idx").setFields(idxFields).setIndexType(null);
+
+        List<QueryIndex> idxs = new ArrayList<>();
+
+        idxs.add(idx);
+
+        entity.setIndexes(idxs);
+
+        ccfg.setQueryEntities(Collections.singleton(entity));
+
+        GridTestUtils.assertThrows(log, new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                grid(0).createCache(ccfg);
+
+                return null;
+            }
+        }, IgniteCheckedException.class, "Index type is not set");
+    }
+
+    /**
+     * Test duplicated index name.
+     *
+     * @throws Exception If failed.
+     */
+    public void testIndexNameDuplicate() throws Exception {
+        final CacheConfiguration ccfg = new CacheConfiguration().setName(CACHE_NAME);
+
+        QueryEntity entity = new QueryEntity();
+
+        entity.setKeyType("Key");
+        entity.setValueType("Value");
+
+        LinkedHashMap<String, String> fields = new LinkedHashMap<>();
+
+        fields.put("a", Integer.class.getName());
+        fields.put("b", Integer.class.getName());
+
+        entity.setFields(fields);
+
+        LinkedHashMap<String, Boolean> idx1Fields = new LinkedHashMap<>();
+        LinkedHashMap<String, Boolean> idx2Fields = new LinkedHashMap<>();
+
+        idx1Fields.put("a", true);
+        idx1Fields.put("b", true);
+
+        QueryIndex idx1 = new QueryIndex().setName("idx").setFields(idx1Fields);
+        QueryIndex idx2 = new QueryIndex().setName("idx").setFields(idx2Fields);
+
+        List<QueryIndex> idxs = new ArrayList<>();
+
+        idxs.add(idx1);
+        idxs.add(idx2);
+
+        entity.setIndexes(idxs);
+
+        ccfg.setQueryEntities(Collections.singleton(entity));
+
+        GridTestUtils.assertThrows(log, new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                grid(0).createCache(ccfg);
+
+                return null;
+            }
+        }, IgniteCheckedException.class, "Duplicate index name");
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/1968e4f5/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
index 405e1f6..862d1a2 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
@@ -93,6 +93,7 @@ import org.apache.ignite.internal.processors.cache.index.DynamicIndexServerCoord
 import org.apache.ignite.internal.processors.cache.index.DynamicIndexServerNodeFIlterBasicSelfTest;
 import org.apache.ignite.internal.processors.cache.index.DynamicIndexServerNodeFilterCoordinatorBasicSelfTest;
 import org.apache.ignite.internal.processors.cache.index.DynamicIndexServerBasicSelfTest;
+import org.apache.ignite.internal.processors.cache.index.QueryEntityValidationSelfTest;
 import org.apache.ignite.internal.processors.cache.index.SchemaExchangeSelfTest;
 import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalAtomicQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalFieldsQuerySelfTest;
@@ -130,6 +131,8 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
         IgniteTestSuite suite = new IgniteTestSuite("Ignite Cache Queries Test Suite");
 
         // Misc tests.
+        // TODO: Enable when IGNITE-1094 is fixed.
+        // suite.addTest(new TestSuite(QueryEntityValidationSelfTest.class));
         suite.addTest(new TestSuite(DuplicateKeyValueClassesSelfTest.class));
 
         // Dynamic index create/drop tests.