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/03/30 11:50:12 UTC

[43/50] [abbrv] ignite git commit: IGNITE-4141 - JDBC driver should always set withKeepBinary flag when querying cache. This fixes #1617.

IGNITE-4141 - JDBC driver should always set withKeepBinary flag when querying cache. This fixes #1617.


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

Branch: refs/heads/ignite-3477-master
Commit: 8dd88d81cf2174fbde920ad00c892995cf913711
Parents: caa16b3
Author: Evgenii Zhuravlev <ez...@gridgain.com>
Authored: Wed Mar 29 15:28:20 2017 -0700
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Wed Mar 29 15:28:20 2017 -0700

----------------------------------------------------------------------
 .../jdbc/AbstractJdbcPojoQuerySelfTest.java     | 169 +++++++++++++++++++
 .../jdbc/JdbcPojoLegacyQuerySelfTest.java       |  44 +++++
 .../ignite/jdbc/JdbcPojoQuerySelfTest.java      |  56 ++++++
 .../jdbc/suite/IgniteJdbcDriverTestSuite.java   |   4 +
 .../ignite/internal/jdbc2/JdbcQueryTask.java    |   2 +-
 .../ignite/internal/jdbc2/JdbcQueryTaskV2.java  |   2 +-
 .../query/jdbc/GridCacheQueryJdbcTask.java      |   4 +-
 7 files changed, 277 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/8dd88d81/modules/clients/src/test/java/org/apache/ignite/jdbc/AbstractJdbcPojoQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/AbstractJdbcPojoQuerySelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/AbstractJdbcPojoQuerySelfTest.java
new file mode 100644
index 0000000..a150574
--- /dev/null
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/AbstractJdbcPojoQuerySelfTest.java
@@ -0,0 +1,169 @@
+/*
+ * 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.jdbc;
+
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.Statement;
+import java.util.Collections;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.binary.BinaryObjectBuilder;
+import org.apache.ignite.cache.QueryEntity;
+import org.apache.ignite.cache.QueryIndex;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.ConnectorConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.binary.BinaryMarshaller;
+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.config.GridTestProperties;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+
+/**
+ * Test for Jdbc driver query without class on client
+ */
+public abstract class AbstractJdbcPojoQuerySelfTest extends GridCommonAbstractTest {
+    /** IP finder. */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** TestObject class name. */
+    protected static final String TEST_OBJECT = "org.apache.ignite.internal.JdbcTestObject";
+
+    /** TestObject class name. */
+    protected static final String TEST_OBJECT_2 = "org.apache.ignite.internal.JdbcTestObject2";
+
+    /** Statement. */
+    protected Statement stmt;
+
+    /** */
+    private String marshallerBackup = GridTestProperties.getProperty(GridTestProperties.MARSH_CLASS_NAME);
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        GridTestProperties.setProperty(GridTestProperties.MARSH_CLASS_NAME, BinaryMarshaller.class.getName());
+
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        CacheConfiguration<?,?> cache = defaultCacheConfiguration();
+
+        cache.setWriteSynchronizationMode(FULL_SYNC);
+        cache.setAtomicityMode(TRANSACTIONAL);
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(IP_FINDER);
+
+        cfg.setDiscoverySpi(disco);
+        cfg.setConnectorConfiguration(new ConnectorConfiguration());
+
+        QueryEntity queryEntity = new QueryEntity();
+        queryEntity.setKeyType("java.lang.String");
+        queryEntity.setValueType("org.apache.ignite.internal.JdbcTestObject");
+        queryEntity.addQueryField("id", "java.lang.Integer", null);
+        queryEntity.addQueryField("testObject", "org.apache.ignite.internal.JdbcTestObject2", null);
+        queryEntity.setIndexes(Collections.singletonList(new QueryIndex("id")));
+
+        cache.setQueryEntities(Collections.singletonList(queryEntity));
+
+        cfg.setCacheConfiguration(cache);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        Ignite ignite = startGrid(0);
+
+        BinaryObjectBuilder builder = ignite.binary().builder(TEST_OBJECT);
+        BinaryObjectBuilder builder2 = ignite.binary().builder(TEST_OBJECT_2);
+
+        builder2.setField("id", 1);
+        builder2.setField("boolVal", true);
+
+        BinaryObject testObject = builder2.build();
+
+        builder.setField("id", 1);
+        builder.setField("testObject", testObject);
+
+        BinaryObject binObj = builder.build();
+
+        IgniteCache<String, BinaryObject> cache = grid(0).cache(null);
+
+        cache.put("0", binObj);
+
+        Class.forName("org.apache.ignite.IgniteJdbcDriver");
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+
+        GridTestProperties.setProperty(GridTestProperties.MARSH_CLASS_NAME, marshallerBackup);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        stmt = DriverManager.getConnection(getURL()).createStatement();
+
+        assertNotNull(stmt);
+        assertFalse(stmt.isClosed());
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        if (stmt != null) {
+            stmt.getConnection().close();
+            stmt.close();
+
+            assertTrue(stmt.isClosed());
+        }
+    }
+
+    /**
+     * @param rs Result set.
+     * @throws Exception In case of error.
+     */
+    protected void assertResultSet(ResultSet rs) throws Exception {
+        assertNotNull(rs);
+
+        int cnt = 0;
+
+        while (rs.next()) {
+            assertNotNull(rs.getString("id"));
+            assertNotNull(rs.getString("testObject"));
+
+            assertTrue(rs.getObject("testObject").toString().contains("id=1"));
+            assertTrue(rs.getObject("testObject").toString().contains("boolVal=true"));
+
+            cnt++;
+        }
+
+        assertEquals(1, cnt);
+    }
+
+    /**
+     * @return URL.
+     */
+    protected abstract String getURL();
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8dd88d81/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcPojoLegacyQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcPojoLegacyQuerySelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcPojoLegacyQuerySelfTest.java
new file mode 100644
index 0000000..4fa7ba5
--- /dev/null
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcPojoLegacyQuerySelfTest.java
@@ -0,0 +1,44 @@
+/*
+ * 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.jdbc;
+
+import java.sql.ResultSet;
+
+/**
+ * Test for Jdbc driver query without class on client
+ */
+public class JdbcPojoLegacyQuerySelfTest extends AbstractJdbcPojoQuerySelfTest {
+    /** URL. */
+    private static final String URL = "jdbc:ignite://127.0.0.1/";
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testJdbcQuery() throws Exception {
+        stmt.execute("select * from JdbcTestObject");
+
+        ResultSet rs = stmt.getResultSet();
+
+        assertResultSet(rs);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected String getURL() {
+        return URL;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8dd88d81/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcPojoQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcPojoQuerySelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcPojoQuerySelfTest.java
new file mode 100644
index 0000000..c2af8a1
--- /dev/null
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcPojoQuerySelfTest.java
@@ -0,0 +1,56 @@
+/*
+ * 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.jdbc;
+
+import java.sql.ResultSet;
+
+import static org.apache.ignite.IgniteJdbcDriver.CFG_URL_PREFIX;
+
+/**
+ * Test for Jdbc driver query without class on client
+ */
+public class JdbcPojoQuerySelfTest extends AbstractJdbcPojoQuerySelfTest {
+    /** URL. */
+    private static final String URL = CFG_URL_PREFIX + "modules/clients/src/test/config/jdbc-bin-config.xml";
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testJdbcQueryTask2() throws Exception {
+        stmt.execute("select * from JdbcTestObject");
+
+        ResultSet rs = stmt.getResultSet();
+
+        assertResultSet(rs);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testJdbcQueryTask1() throws Exception {
+        ResultSet rs = stmt.executeQuery("select * from JdbcTestObject");
+
+        assertResultSet(rs);
+
+    }
+
+    /** {@inheritDoc} */
+    @Override protected String getURL() {
+        return URL;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8dd88d81/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java
index 85e7d90..2489de9 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java
@@ -24,6 +24,8 @@ import org.apache.ignite.jdbc.JdbcEmptyCacheSelfTest;
 import org.apache.ignite.jdbc.JdbcLocalCachesSelfTest;
 import org.apache.ignite.jdbc.JdbcMetadataSelfTest;
 import org.apache.ignite.jdbc.JdbcNoDefaultCacheTest;
+import org.apache.ignite.jdbc.JdbcPojoLegacyQuerySelfTest;
+import org.apache.ignite.jdbc.JdbcPojoQuerySelfTest;
 import org.apache.ignite.jdbc.JdbcPreparedStatementSelfTest;
 import org.apache.ignite.jdbc.JdbcResultSetSelfTest;
 import org.apache.ignite.jdbc.JdbcStatementSelfTest;
@@ -49,6 +51,8 @@ public class IgniteJdbcDriverTestSuite extends TestSuite {
         suite.addTest(new TestSuite(JdbcEmptyCacheSelfTest.class));
         suite.addTest(new TestSuite(JdbcLocalCachesSelfTest.class));
         suite.addTest(new TestSuite(JdbcNoDefaultCacheTest.class));
+        suite.addTest(new TestSuite(JdbcPojoQuerySelfTest.class));
+        suite.addTest(new TestSuite(JdbcPojoLegacyQuerySelfTest.class));
 
         // Ignite client node based driver tests
         suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcConnectionSelfTest.class));

http://git-wip-us.apache.org/repos/asf/ignite/blob/8dd88d81/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryTask.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryTask.java
index 0b23f9b..bd6b0f2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryTask.java
@@ -158,7 +158,7 @@ class JdbcQueryTask implements IgniteCallable<JdbcQueryTask.QueryResult> {
             qry.setCollocated(collocatedQry);
             qry.setDistributedJoins(distributedJoins);
 
-            QueryCursor<List<?>> qryCursor = cache.query(qry);
+            QueryCursor<List<?>> qryCursor = cache.withKeepBinary().query(qry);
 
             Collection<GridQueryFieldMetadata> meta = ((QueryCursorImpl<List<?>>)qryCursor).fieldsMeta();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/8dd88d81/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryTaskV2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryTaskV2.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryTaskV2.java
index 9093d15..61f152d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryTaskV2.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryTaskV2.java
@@ -161,7 +161,7 @@ class JdbcQueryTaskV2 implements IgniteCallable<JdbcQueryTaskV2.QueryResult> {
             qry.setCollocated(collocatedQry);
             qry.setDistributedJoins(distributedJoins);
 
-            QueryCursorImpl<List<?>> qryCursor = (QueryCursorImpl<List<?>>)cache.query(qry);
+            QueryCursorImpl<List<?>> qryCursor = (QueryCursorImpl<List<?>>)cache.withKeepBinary().query(qry);
 
             if (isQry == null)
                 isQry = qryCursor.isQuery();

http://git-wip-us.apache.org/repos/asf/ignite/blob/8dd88d81/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/jdbc/GridCacheQueryJdbcTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/jdbc/GridCacheQueryJdbcTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/jdbc/GridCacheQueryJdbcTask.java
index ca08ead..4ae8a8a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/jdbc/GridCacheQueryJdbcTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/jdbc/GridCacheQueryJdbcTask.java
@@ -268,7 +268,7 @@ public class GridCacheQueryJdbcTask extends ComputeTaskAdapter<byte[], byte[]> {
 
                 qry.setPageSize(pageSize);
 
-                QueryCursor<List<?>> cursor = cache.query(qry);
+                QueryCursor<List<?>> cursor = cache.withKeepBinary().query(qry);
 
                 Collection<GridQueryFieldMetadata> meta = ((QueryCursorImpl<List<?>>)cursor).fieldsMeta();
 
@@ -453,4 +453,4 @@ public class GridCacheQueryJdbcTask extends ComputeTaskAdapter<byte[], byte[]> {
             return iter;
         }
     }
-}
\ No newline at end of file
+}