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 E7362200BF0 for ; Thu, 24 Nov 2016 14:24:15 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id E5D4D160B11; Thu, 24 Nov 2016 13:24:15 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 2AD4E160B31 for ; Thu, 24 Nov 2016 14:24:13 +0100 (CET) Received: (qmail 17697 invoked by uid 500); 24 Nov 2016 13:24:12 -0000 Mailing-List: contact commits-help@ignite.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@ignite.apache.org Delivered-To: mailing list commits@ignite.apache.org Received: (qmail 17268 invoked by uid 99); 24 Nov 2016 13:24:12 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 24 Nov 2016 13:24:12 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 2930CE055D; Thu, 24 Nov 2016 13:24:12 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: av@apache.org To: commits@ignite.apache.org Date: Thu, 24 Nov 2016 13:24:31 -0000 Message-Id: In-Reply-To: <60e3a79c717a4fc68f717e1a06f724c3@git.apache.org> References: <60e3a79c717a4fc68f717e1a06f724c3@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [21/50] [abbrv] ignite git commit: Backport commit of the following: archived-at: Thu, 24 Nov 2016 13:24:16 -0000 Backport commit of the following: commit 612eb3daffe608995aac28eed019b3e6ef9d66d3 Author: Aleksei Scherbakov Date: Fri Aug 19 13:28:39 2016 +0300 ignite-2795 Support 'copyOnRead' for SQL queries Project: http://git-wip-us.apache.org/repos/asf/ignite/repo Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/8b59f4e7 Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/8b59f4e7 Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/8b59f4e7 Branch: refs/heads/ignite-4242 Commit: 8b59f4e76138e08e80aa219c1a9cf0c3df6fdb4b Parents: 26daa57 Author: Andrey V. Mashenkov Authored: Thu Nov 10 14:43:00 2016 +0300 Committer: Andrey V. Mashenkov Committed: Thu Nov 10 14:43:00 2016 +0300 ---------------------------------------------------------------------- .../processors/cache/GridCacheContext.java | 8 + .../query/h2/GridH2ResultSetIterator.java | 62 ++++- .../processors/query/h2/IgniteH2Indexing.java | 4 +- .../query/h2/opt/GridH2ValueCacheObject.java | 10 +- .../query/h2/twostep/GridMapQueryExecutor.java | 34 ++- .../h2/twostep/GridReduceQueryExecutor.java | 2 +- .../cache/CacheSqlQueryValueCopySelfTest.java | 226 +++++++++++++++++++ .../IgniteCacheQuerySelfTestSuite2.java | 2 + 8 files changed, 338 insertions(+), 10 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/8b59f4e7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java index dc9c766..30f1c4b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java @@ -1699,6 +1699,14 @@ public class GridCacheContext implements Externalizable { } /** + * @return {@code True} if the value for the cache object has to be copied because + * of {@link CacheConfiguration#isCopyOnRead()}. + */ + public boolean needValueCopy() { + return affNode && cacheCfg.isCopyOnRead() && cacheCfg.getMemoryMode() != OFFHEAP_VALUES; + } + + /** * Converts temporary offheap object to heap-based. * * @param obj Object. http://git-wip-us.apache.org/repos/asf/ignite/blob/8b59f4e7/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/GridH2ResultSetIterator.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/GridH2ResultSetIterator.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/GridH2ResultSetIterator.java index 3603bb5..e0680d3 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/GridH2ResultSetIterator.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/GridH2ResultSetIterator.java @@ -17,24 +17,49 @@ package org.apache.ignite.internal.processors.query.h2; +import java.lang.reflect.Field; import java.sql.ResultSet; import java.sql.SQLException; import java.util.NoSuchElementException; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.query.h2.opt.GridH2ValueCacheObject; import org.apache.ignite.internal.util.GridCloseableIteratorAdapter; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; - +import org.h2.jdbc.JdbcResultSet; +import org.h2.result.ResultInterface; +import org.h2.value.Value; /** * Iterator over result set. */ public abstract class GridH2ResultSetIterator extends GridCloseableIteratorAdapter { /** */ + private static final Field RESULT_FIELD; + + /** + * Initialize. + */ + static { + try { + RESULT_FIELD = JdbcResultSet.class.getDeclaredField("result"); + + RESULT_FIELD.setAccessible(true); + } + catch (NoSuchFieldException e) { + throw new IllegalStateException("Check H2 version in classpath.", e); + } + } + + /** */ private static final long serialVersionUID = 0L; /** */ + private final ResultInterface res; + + /** */ private final ResultSet data; /** */ @@ -49,12 +74,20 @@ public abstract class GridH2ResultSetIterator extends GridCloseableIteratorAd /** * @param data Data array. * @param closeStmt If {@code true} closes result set statement when iterator is closed. + * @param needCpy {@code True} if need copy cache object's value. * @throws IgniteCheckedException If failed. */ - protected GridH2ResultSetIterator(ResultSet data, boolean closeStmt) throws IgniteCheckedException { + protected GridH2ResultSetIterator(ResultSet data, boolean closeStmt, boolean needCpy) throws IgniteCheckedException { this.data = data; this.closeStmt = closeStmt; + try { + res = needCpy ? (ResultInterface)RESULT_FIELD.get(data) : null; + } + catch (IllegalAccessException e) { + throw new IllegalStateException(e); // Must not happen. + } + if (data != null) { try { row = new Object[data.getMetaData().getColumnCount()]; @@ -78,8 +111,27 @@ public abstract class GridH2ResultSetIterator extends GridCloseableIteratorAd if (!data.next()) return false; - for (int c = 0; c < row.length; c++) - row[c] = data.getObject(c + 1); + if (res != null) { + Value[] values = res.currentRow(); + + for (int c = 0; c < row.length; c++) { + Value val = values[c]; + + if (val instanceof GridH2ValueCacheObject) { + GridH2ValueCacheObject valCacheObj = (GridH2ValueCacheObject)values[c]; + + GridCacheContext cctx = valCacheObj.getCacheContext(); + + row[c] = valCacheObj.getObject(cctx != null && cctx.needValueCopy()); + } + else + row[c] = val.getObject(); + } + } + else { + for (int c = 0; c < row.length; c++) + row[c] = data.getObject(c + 1); + } return true; } @@ -134,6 +186,6 @@ public abstract class GridH2ResultSetIterator extends GridCloseableIteratorAd /** {@inheritDoc} */ @Override public String toString() { - return S.toString((Class)getClass(), this); + return S.toString(GridH2ResultSetIterator.class, this); } } \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/8b59f4e7/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 ab332c1..5c2fab5 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 @@ -2230,7 +2230,7 @@ public class IgniteH2Indexing implements GridQueryIndexing { * @throws IgniteCheckedException If failed. */ protected FieldsIterator(ResultSet data) throws IgniteCheckedException { - super(data, false); + super(data, false, true); } /** {@inheritDoc} */ @@ -2255,7 +2255,7 @@ public class IgniteH2Indexing implements GridQueryIndexing { * @throws IgniteCheckedException If failed. */ protected KeyValIterator(ResultSet data) throws IgniteCheckedException { - super(data, false); + super(data, false, true); } /** {@inheritDoc} */ http://git-wip-us.apache.org/repos/asf/ignite/blob/8b59f4e7/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2ValueCacheObject.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2ValueCacheObject.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2ValueCacheObject.java index fd0e6ed..4d83295 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2ValueCacheObject.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2ValueCacheObject.java @@ -120,7 +120,15 @@ public class GridH2ValueCacheObject extends Value { /** {@inheritDoc} */ @Override public Object getObject() { - return obj.isPlatformType() ? obj.value(objectContext(), false) : obj; + return getObject(false); + } + + /** + * @param cpy Copy flag. + * @return Value. + */ + public Object getObject(boolean cpy) { + return obj.isPlatformType() ? obj.value(objectContext(), cpy) : obj; } /** {@inheritDoc} */ http://git-wip-us.apache.org/repos/asf/ignite/blob/8b59f4e7/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java index 1f05bf7..7e4d5b6 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java @@ -52,6 +52,7 @@ import org.apache.ignite.internal.processors.cache.query.CacheQueryType; import org.apache.ignite.internal.processors.cache.query.GridCacheSqlQuery; import org.apache.ignite.internal.processors.query.GridQueryCancel; import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing; +import org.apache.ignite.internal.processors.query.h2.opt.GridH2ValueCacheObject; import org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryCancelRequest; import org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryFailResponse; import org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryNextPageRequest; @@ -759,6 +760,9 @@ public class GridMapQueryExecutor { private final int rowCount; /** */ + private boolean cpNeeded; + + /** */ private volatile boolean closed; /** @@ -767,11 +771,12 @@ public class GridMapQueryExecutor { * @param qrySrcNodeId Query source node. * @param qry Query. */ - private QueryResult(ResultSet rs, GridCacheContext cctx, UUID qrySrcNodeId, GridCacheSqlQuery qry) { + private QueryResult(ResultSet rs, GridCacheContext cctx, UUID qrySrcNodeId, GridCacheSqlQuery qry) { this.rs = rs; this.cctx = cctx; this.qry = qry; this.qrySrcNodeId = qrySrcNodeId; + this.cpNeeded = cctx.isLocalNode(qrySrcNodeId); try { res = (ResultInterface)RESULT_FIELD.get(rs); @@ -803,6 +808,33 @@ public class GridMapQueryExecutor { Value[] row = res.currentRow(); + if (cpNeeded) { + boolean copied = false; + + for (int j = 0; j < row.length; j++) { + Value val = row[j]; + + if (val instanceof GridH2ValueCacheObject) { + GridH2ValueCacheObject valCacheObj = (GridH2ValueCacheObject)val; + + GridCacheContext cctx = valCacheObj.getCacheContext(); + + if (cctx != null && cctx.needValueCopy()) { + row[j] = new GridH2ValueCacheObject(valCacheObj.getCacheContext(), valCacheObj.getCacheObject()) { + @Override public Object getObject() { + return getObject(true); + } + }; + + copied = true; + } + } + } + + if (i == 0 && !copied) + cpNeeded = false; // No copy on read caches, skip next checks. + } + assert row != null; if (readEvt) { http://git-wip-us.apache.org/repos/asf/ignite/blob/8b59f4e7/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java index 3fdbf42..29ad94a 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java @@ -1289,7 +1289,7 @@ public class GridReduceQueryExecutor { * @throws IgniteCheckedException If failed. */ protected Iter(ResultSet data) throws IgniteCheckedException { - super(data, true); + super(data, true, false); } /** {@inheritDoc} */ http://git-wip-us.apache.org/repos/asf/ignite/blob/8b59f4e7/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheSqlQueryValueCopySelfTest.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheSqlQueryValueCopySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheSqlQueryValueCopySelfTest.java new file mode 100644 index 0000000..e47e893 --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheSqlQueryValueCopySelfTest.java @@ -0,0 +1,226 @@ +/* + * 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; + +import java.util.List; +import javax.cache.Cache; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.query.QueryCursor; +import org.apache.ignite.cache.query.SqlFieldsQuery; +import org.apache.ignite.cache.query.SqlQuery; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; +import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * Tests modification of values returned by query iterators with enabled copy on read. + */ +public class CacheSqlQueryValueCopySelfTest extends GridCommonAbstractTest { + /** */ + private static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true); + + /** */ + private static final int KEYS = 100; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + if ("client".equals(cfg.getGridName())) + cfg.setClientMode(true); + + ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder); + + CacheConfiguration cc = new CacheConfiguration<>(); + + cc.setCopyOnRead(true); + cc.setIndexedTypes(Integer.class, Value.class); + + cfg.setCacheConfiguration(cc); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + super.beforeTestsStarted(); + + startGridsMultiThreaded(3); + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + IgniteCache cache = grid(0).cache(null); + + for (int i = 0; i < KEYS; i++) + cache.put(i, new Value("before")); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + IgniteCache cache = grid(0).cache(null); + + cache.removeAll(); + + super.afterTest(); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + super.afterTestsStopped(); + + stopAllGrids(); + } + + /** + * Tests two step query from dedicated client. + * + * @throws Exception If failed. + */ + public void testTwoStepSqlClientQuery() throws Exception { + try (Ignite client = startGrid("client")) { + IgniteCache cache = client.cache(null); + + List> all = cache.query( + new SqlQuery(Value.class, "select * from Value")).getAll(); + + assertEquals(KEYS, all.size()); + + for (Cache.Entry entry : all) + entry.getValue().str = "after"; + + check(cache); + + QueryCursor> qry = cache.query(new SqlFieldsQuery("select _val from Value")); + + List> all0 = qry.getAll(); + + assertEquals(KEYS, all0.size()); + + for (List entry : all0) + ((Value)entry.get(0)).str = "after"; + + check(cache); + } + } + + /** + * Test two step query without local reduce phase. + */ + public void testTwoStepSkipReduceSqlQuery() { + IgniteCache cache = grid(0).cache(null); + + List> all = cache.query( + new SqlQuery(Value.class, "select * from Value").setPageSize(3)).getAll(); + + assertEquals(KEYS, all.size()); + + for (Cache.Entry entry : all) + entry.getValue().str = "after"; + + check(cache); + } + + /** + * Test two step query value copy. + */ + public void testTwoStepReduceSqlQuery() { + IgniteCache cache = grid(0).cache(null); + + QueryCursor> qry = cache.query(new SqlFieldsQuery("select _val from Value order by _key")); + + List> all = qry.getAll(); + + assertEquals(KEYS, all.size()); + + for (List entry : all) + ((Value)entry.get(0)).str = "after"; + + check(cache); + } + + /** + * Tests local sql query. + */ + public void testLocalSqlQuery() { + IgniteCache cache = grid(0).cache(null); + + SqlQuery qry = new SqlQuery<>(Value.class.getSimpleName(), "select * from Value"); + qry.setLocal(true); + + List> all = cache.query(qry).getAll(); + + assertFalse(all.isEmpty()); + + for (Cache.Entry entry : all) + entry.getValue().str = "after"; + + check(cache); + } + + /** + * Tests local sql query. + */ + public void testLocalSqlFieldsQuery() { + IgniteCache cache = grid(0).cache(null); + + QueryCursor> cur = cache.query(new SqlFieldsQuery("select _val from Value").setLocal(true)); + + List> all = cur.getAll(); + + assertFalse(all.isEmpty()); + + for (List entry : all) + ((Value)entry.get(0)).str = "after"; + + check(cache); + } + + /** */ + private static class Value { + /** */ + private String str; + + /** + * @param str String. + */ + public Value(String str) { + this.str = str; + } + } + + /** + * @param cache Cache. + */ + private void check(IgniteCache cache) { + int cnt = 0; + + // Value should be not modified by previous assignment. + for (Cache.Entry entry : cache) { + cnt++; + + assertEquals("before", entry.getValue().str); + } + + assertEquals(KEYS, cnt); + } +} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/8b59f4e7/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java ---------------------------------------------------------------------- diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java index 9128f76..4b4a576 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java +++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java @@ -27,6 +27,7 @@ import org.apache.ignite.internal.processors.cache.CacheQueryOffheapEvictDataLos import org.apache.ignite.internal.processors.cache.CacheReplicatedQueryMetricsDistributedSelfTest; import org.apache.ignite.internal.processors.cache.CacheReplicatedQueryMetricsLocalSelfTest; import org.apache.ignite.internal.processors.cache.CacheScanPartitionQueryFallbackSelfTest; +import org.apache.ignite.internal.processors.cache.CacheSqlQueryValueCopySelfTest; import org.apache.ignite.internal.processors.cache.GridCacheQueryIndexingDisabledSelfTest; import org.apache.ignite.internal.processors.cache.IgniteCacheFieldsQueryNoDataSelfTest; import org.apache.ignite.internal.processors.cache.IgniteCacheNoClassQuerySelfTest; @@ -114,6 +115,7 @@ public class IgniteCacheQuerySelfTestSuite2 extends TestSuite { // Other. suite.addTestSuite(CacheQueryNewClientSelfTest.class); suite.addTestSuite(CacheOffheapBatchIndexingSingleTypeTest.class); + suite.addTestSuite(CacheSqlQueryValueCopySelfTest.class); return suite; }