Sergey Kozlov created IGNITE-4334:
-------------------------------------

             Summary: DML: INSERT INTO SELECT FROM statement fails if copy from 
partitioned to replicated cache
                 Key: IGNITE-4334
                 URL: https://issues.apache.org/jira/browse/IGNITE-4334
             Project: Ignite
          Issue Type: Bug
          Components: general
    Affects Versions: 1.8
            Reporter: Sergey Kozlov
             Fix For: 1.8


INSERT INTO SELECT FROM statement fails if executed on a replicated cache and 
used partitioned cache as data source:

{code:title=test.java|borderStyle=solid}
    public static void main(String[] args) throws Exception {
        Ignition.setClientMode(true);

        final Long PRELOAD_NUM = 1_000L;
        final Long RANGE_NUM = 2*PRELOAD_NUM;
        final int OPERATION_LIMIT = 500;

        ArrayList<String> fields = new ArrayList<>(
                Arrays.asList("intCol", "longCol", "shortCol", "stringCol", 
"doubleCol", "bigDecimalCol", "booleanCol", "mapCol")
        );

        List<String> cacheNames = Arrays.asList(
                "tx-part-full-sync",
                "tx-repl-full-sync",
        );

        List<String> cacheTxNames = new ArrayList<String>();

        for (int i=0; i < cacheNames.size(); i++) {
            if (cacheNames.get(i).startsWith("tx"))
                cacheTxNames.add(cacheNames.get(i));
        }
        String srcCache = null;
        try (Ignite ig = Ignition.start("examples/config/ext-sql.xml")) {

            for (String cacheName : cacheNames) {

                ig.cache(cacheName).query(new SqlFieldsQuery("delete from 
AllTypes"));

                if (srcCache == null) {
                    System.out.println("The cache size " + cacheName + ": " + 
ig.cache(cacheName).sizeLong());
                    for (long i = 0L; i < PRELOAD_NUM; i=i+50) {

                        ArrayList<Long> sqlArgs = new ArrayList<Long>();
                        for (int j = 0; j < 50; j++) {
                            sqlArgs.add(i + j);
                        }

                        ig.cache(cacheName).query(
                                makeInsertSqlAndArgsByKey(sqlArgs, false)
                        );

                        if (i > 0 && i % (PRELOAD_NUM / 10) == 0) {
                            Date d = new Date();
                            System.out.println(d.toString() + " Done: " + i);
                            Object val = ig.cache(cacheName).get(i);
                        }
                    }
                    srcCache = cacheName;
                }
                else {
                    ig.cache(cacheName).query(new SqlFieldsQuery("insert into 
AllTypes (_key, _val) select _key, _val from \"" + srcCache + "\".AllTypes"));
                }
                System.out.println("The cache size " + cacheName + ": " + 
ig.cache(cacheName).sizeLong());
            }

        }
{code}

{noformat}
Exception in thread "main" javax.cache.CacheException: Queries running on 
replicated cache should not contain JOINs with partitioned tables 
[rCache=tx-repl-full-sync, pCache=tx-part-full-sync]
        at 
org.apache.ignite.internal.processors.query.h2.twostep.GridReduceQueryExecutor.stableDataNodes(GridReduceQueryExecutor.java:432)
        at 
org.apache.ignite.internal.processors.query.h2.twostep.GridReduceQueryExecutor.query(GridReduceQueryExecutor.java:529)
        at 
org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing$5.iterator(IgniteH2Indexing.java:1119)
        at 
org.apache.ignite.internal.processors.cache.QueryCursorImpl.iterator(QueryCursorImpl.java:98)
        at 
org.apache.ignite.internal.processors.query.h2.DmlStatementsProcessor.doInsert(DmlStatementsProcessor.java:700)
        at 
org.apache.ignite.internal.processors.query.h2.DmlStatementsProcessor.executeUpdateStatement(DmlStatementsProcessor.java:282)
        at 
org.apache.ignite.internal.processors.query.h2.DmlStatementsProcessor.updateSqlFields(DmlStatementsProcessor.java:155)
        at 
org.apache.ignite.internal.processors.query.h2.DmlStatementsProcessor.updateSqlFieldsTwoStep(DmlStatementsProcessor.java:185)
        at 
org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing.queryTwoStep(IgniteH2Indexing.java:1266)
        at 
org.apache.ignite.internal.processors.query.GridQueryProcessor$4.applyx(GridQueryProcessor.java:812)
        at 
org.apache.ignite.internal.processors.query.GridQueryProcessor$4.applyx(GridQueryProcessor.java:810)
        at 
org.apache.ignite.internal.util.lang.IgniteOutClosureX.apply(IgniteOutClosureX.java:36)
        at 
org.apache.ignite.internal.processors.query.GridQueryProcessor.executeQuery(GridQueryProcessor.java:1777)
        at 
org.apache.ignite.internal.processors.query.GridQueryProcessor.queryTwoStep(GridQueryProcessor.java:810)
        at 
org.apache.ignite.internal.processors.cache.IgniteCacheProxy.query(IgniteCacheProxy.java:749)
        at 
org.apache.ignite.examples.datagrid.ExtSqlExample.main(ExtSqlExample.java:221)
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
        at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.lang.reflect.Method.invoke(Method.java:606)
        at com.intellij.rt.execution.application.AppMain.main(AppMain.java:147)

{noformat}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to