You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2017/08/01 10:37:10 UTC

[10/49] ignite git commit: IGNITE-5767 Web Console: Changed mapping for BINARY SQL type to byte[].

IGNITE-5767 Web Console: Changed mapping for BINARY SQL type to byte[].


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

Branch: refs/heads/ignite-5578
Commit: 995258f9a326bb5a08b1e004d92e2760c25f20c0
Parents: b49469f
Author: Vasiliy Sisko <vs...@gridgain.com>
Authored: Thu Jul 27 11:06:52 2017 +0700
Committer: Andrey Novikov <an...@gridgain.com>
Committed: Thu Jul 27 11:06:52 2017 +0700

----------------------------------------------------------------------
 .../store/jdbc/CacheJdbcPojoStoreTest.java      |  55 ++++++++-
 .../cache/store/jdbc/model/BinaryTest.java      | 122 +++++++++++++++++++
 .../cache/store/jdbc/model/BinaryTestKey.java   |  87 +++++++++++++
 .../frontend/app/data/jdbc-types.json           |   6 +-
 4 files changed, 264 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/995258f9/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreTest.java
index bb85cab..ea2808f 100644
--- a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreTest.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.cache.store.jdbc;
 
+import java.io.ByteArrayInputStream;
 import java.lang.reflect.Field;
 import java.sql.Connection;
 import java.sql.PreparedStatement;
@@ -25,6 +26,7 @@ import java.sql.Statement;
 import java.sql.Timestamp;
 import java.sql.Types;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentLinkedQueue;
@@ -33,6 +35,8 @@ import org.apache.ignite.Ignite;
 import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.binary.BinaryObjectBuilder;
 import org.apache.ignite.cache.store.jdbc.dialect.H2Dialect;
+import org.apache.ignite.cache.store.jdbc.model.BinaryTest;
+import org.apache.ignite.cache.store.jdbc.model.BinaryTestKey;
 import org.apache.ignite.cache.store.jdbc.model.Organization;
 import org.apache.ignite.cache.store.jdbc.model.OrganizationKey;
 import org.apache.ignite.cache.store.jdbc.model.Person;
@@ -44,7 +48,6 @@ import org.apache.ignite.internal.util.typedef.CI2;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiInClosure;
 import org.apache.ignite.testframework.GridTestUtils;
-import org.apache.ignite.testframework.config.GridTestProperties;
 import org.apache.ignite.testframework.junits.cache.GridAbstractCacheStoreSelfTest;
 import org.h2.jdbcx.JdbcConnectionPool;
 
@@ -78,7 +81,7 @@ public class CacheJdbcPojoStoreTest extends GridAbstractCacheStoreSelfTest<Cache
     @Override protected CacheJdbcPojoStore<Object, Object> store() {
         CacheJdbcPojoStoreFactory<Object, Object> storeFactory = new CacheJdbcPojoStoreFactory<>();
 
-        JdbcType[] storeTypes = new JdbcType[6];
+        JdbcType[] storeTypes = new JdbcType[7];
 
         storeTypes[0] = new JdbcType();
         storeTypes[0].setDatabaseSchema("PUBLIC");
@@ -147,6 +150,15 @@ public class CacheJdbcPojoStoreTest extends GridAbstractCacheStoreSelfTest<Cache
         storeTypes[5].setValueType("java.util.UUID");
         storeTypes[5].setValueFields(new JdbcTypeField(Types.BINARY, "VAL", UUID.class, null));
 
+        storeTypes[6] = new JdbcType();
+        storeTypes[6].setDatabaseSchema("PUBLIC");
+        storeTypes[6].setDatabaseTable("BINARY_ENTRIES");
+        storeTypes[6].setKeyType("org.apache.ignite.cache.store.jdbc.model.BinaryTestKey");
+        storeTypes[6].setKeyFields(new JdbcTypeField(Types.BINARY, "KEY", Integer.class, "id"));
+
+        storeTypes[6].setValueType("org.apache.ignite.cache.store.jdbc.model.BinaryTest");
+        storeTypes[6].setValueFields(new JdbcTypeField(Types.BINARY, "VAL", byte[].class, "bytes"));
+
         storeFactory.setTypes(storeTypes);
 
         storeFactory.setDialect(new H2Dialect());
@@ -210,6 +222,13 @@ public class CacheJdbcPojoStoreTest extends GridAbstractCacheStoreSelfTest<Cache
             // No-op.
         }
 
+        try {
+            stmt.executeUpdate("delete from Binary_Entries");
+        }
+        catch (SQLException ignore) {
+            // No-op.
+        }
+
         stmt.executeUpdate("CREATE TABLE IF NOT EXISTS " +
             "String_Entries (key varchar(100) not null, val varchar(100), PRIMARY KEY(key))");
 
@@ -217,6 +236,9 @@ public class CacheJdbcPojoStoreTest extends GridAbstractCacheStoreSelfTest<Cache
             "UUID_Entries (key binary(16) not null, val binary(16), PRIMARY KEY(key))");
 
         stmt.executeUpdate("CREATE TABLE IF NOT EXISTS " +
+            "Binary_Entries (key binary(16) not null, val binary(16), PRIMARY KEY(key))");
+
+        stmt.executeUpdate("CREATE TABLE IF NOT EXISTS " +
             "Timestamp_Entries (key timestamp not null, val integer, PRIMARY KEY(key))");
 
         stmt.executeUpdate("CREATE TABLE IF NOT EXISTS " +
@@ -304,11 +326,31 @@ public class CacheJdbcPojoStoreTest extends GridAbstractCacheStoreSelfTest<Cache
 
         conn.commit();
 
+        U.closeQuiet(prnStmt);
+
+        PreparedStatement binaryStmt = conn.prepareStatement("INSERT INTO Binary_Entries(key, val) VALUES (?, ?)");
+
+        byte[] bytes = new byte[16];
+
+        for (byte i = 0; i < 16; i++)
+            bytes[i] = i;
+
+        binaryStmt.setInt(1, 1);
+        binaryStmt.setBinaryStream(2, new ByteArrayInputStream(bytes));
+
+        binaryStmt.addBatch();
+        binaryStmt.executeBatch();
+
+        U.closeQuiet(binaryStmt);
+
+        conn.commit();
+
         U.closeQuiet(conn);
 
         final Collection<Object> orgKeys = new ConcurrentLinkedQueue<>();
         final Collection<Object> prnKeys = new ConcurrentLinkedQueue<>();
         final Collection<Object> prnComplexKeys = new ConcurrentLinkedQueue<>();
+        final Collection<Object> binaryTestVals = new ConcurrentLinkedQueue<>();
 
         IgniteBiInClosure<Object, Object> c = new CI2<Object, Object>() {
             @Override public void apply(Object k, Object v) {
@@ -331,12 +373,18 @@ public class CacheJdbcPojoStoreTest extends GridAbstractCacheStoreSelfTest<Cache
                         if (PersonComplexKey.class.getName().equals(keyType)
                             && Person.class.getName().equals(valType))
                             prnComplexKeys.add(key);
+
+                        if (BinaryTestKey.class.getName().equals(keyType)
+                            && BinaryTest.class.getName().equals(valType))
+                            binaryTestVals.add(val.field("bytes"));
                     }
                 }else {
                     if (k instanceof OrganizationKey && v instanceof Organization)
                         orgKeys.add(k);
                     else if (k instanceof PersonKey && v instanceof Person)
                         prnKeys.add(k);
+                    else if (k instanceof BinaryTestKey && v instanceof BinaryTest)
+                        binaryTestVals.add(((BinaryTest)v).getBytes());
                     else if (k instanceof PersonComplexKey && v instanceof Person) {
                         PersonComplexKey key = (PersonComplexKey)k;
 
@@ -357,6 +405,8 @@ public class CacheJdbcPojoStoreTest extends GridAbstractCacheStoreSelfTest<Cache
         assertEquals(ORGANIZATION_CNT, orgKeys.size());
         assertEquals(PERSON_CNT, prnKeys.size());
         assertEquals(PERSON_CNT, prnComplexKeys.size());
+        assertEquals(1, binaryTestVals.size());
+        assertTrue(Arrays.equals(bytes, (byte[])binaryTestVals.iterator().next()));
 
         Collection<Object> tmpOrgKeys = new ArrayList<>(orgKeys);
         Collection<Object> tmpPrnKeys = new ArrayList<>(prnKeys);
@@ -545,5 +595,4 @@ public class CacheJdbcPojoStoreTest extends GridAbstractCacheStoreSelfTest<Cache
 
         return obj;
     }
-
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/995258f9/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/model/BinaryTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/model/BinaryTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/model/BinaryTest.java
new file mode 100644
index 0000000..6cb551b
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/model/BinaryTest.java
@@ -0,0 +1,122 @@
+/*
+ * 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.cache.store.jdbc.model;
+
+import java.io.Serializable;
+import java.sql.Date;
+import java.util.Arrays;
+
+/**
+ * BinaryTest definition.
+ */
+public class BinaryTest implements Serializable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Value for id. */
+    private Integer id;
+
+    /** Value for bytes. */
+    private byte[] bytes;
+
+    /**
+     * Empty constructor.
+     */
+    public BinaryTest() {
+        // No-op.
+    }
+
+    /**
+     * Full constructor.
+     */
+    public BinaryTest(
+        Integer id,
+        byte[] bytes
+    ) {
+        this.id = id;
+        this.bytes = bytes;
+    }
+
+    /**
+     * Gets id.
+     *
+     * @return Value for id.
+     */
+    public Integer getId() {
+        return id;
+    }
+
+    /**
+     * Sets id.
+     *
+     * @param id New value for id.
+     */
+    public void setId(Integer id) {
+        this.id = id;
+    }
+
+    /**
+     * Gets bytes.
+     *
+     * @return Value for bytes.
+     */
+    public byte[] getBytes() {
+        return bytes;
+    }
+
+    /**
+     * Sets bytes.
+     *
+     * @param bytes New value for bytes.
+     */
+    public void setBytes(byte[] bytes) {
+        this.bytes = bytes;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        if (this == o)
+            return true;
+
+        if (!(o instanceof BinaryTest))
+            return false;
+
+        BinaryTest that = (BinaryTest)o;
+
+        if (id != null ? !id.equals(that.id) : that.id != null)
+            return false;
+
+        return bytes != null ? Arrays.equals(bytes, that.bytes) : that.bytes == null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        int res = id != null ? id.hashCode() : 0;
+
+        res = 31 * res + (bytes != null ? Arrays.hashCode(bytes) : 0);
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return "Person [id=" + id +
+            ", bytes=" + Arrays.toString(bytes) +
+            "]";
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/995258f9/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/model/BinaryTestKey.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/model/BinaryTestKey.java b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/model/BinaryTestKey.java
new file mode 100644
index 0000000..b710fad
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/model/BinaryTestKey.java
@@ -0,0 +1,87 @@
+/*
+ * 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.cache.store.jdbc.model;
+
+import java.io.Serializable;
+
+/**
+ * BinaryTestKey definition.
+ */
+public class BinaryTestKey implements Serializable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Value for id. */
+    private Integer id;
+
+    /**
+     * Empty constructor.
+     */
+    public BinaryTestKey() {
+        // No-op.
+    }
+
+    /**
+     * Full constructor.
+     */
+    public BinaryTestKey(Integer id) {
+        this.id = id;
+    }
+
+    /**
+     * Gets id.
+     *
+     * @return Value for id.
+     */
+    public Integer getId() {
+        return id;
+    }
+
+    /**
+     * Sets id.
+     *
+     * @param id New value for id.
+     */
+    public void setId(Integer id) {
+        this.id = id;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        if (this == o)
+            return true;
+
+        if (!(o instanceof BinaryTestKey))
+            return false;
+
+        BinaryTestKey that = (BinaryTestKey)o;
+
+        return id != null ? id.equals(that.id) : that.id == null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        return id != null ? id.hashCode() : 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return "PersonKey [id=" + id +
+            "]";
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/995258f9/modules/web-console/frontend/app/data/jdbc-types.json
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/data/jdbc-types.json b/modules/web-console/frontend/app/data/jdbc-types.json
index 07abbaf..c743e31 100644
--- a/modules/web-console/frontend/app/data/jdbc-types.json
+++ b/modules/web-console/frontend/app/data/jdbc-types.json
@@ -21,9 +21,9 @@
     {"dbName": "DATE", "dbType": 91, "signed": {"javaType": "Date"}},
     {"dbName": "TIME", "dbType": 92, "signed": {"javaType": "Time"}},
     {"dbName": "TIMESTAMP", "dbType": 93, "signed": {"javaType": "Timestamp"}},
-    {"dbName": "BINARY", "dbType": -2, "signed": {"javaType": "Object"}},
-    {"dbName": "VARBINARY", "dbType": -3, "signed": {"javaType": "Object"}},
-    {"dbName": "LONGVARBINARY", "dbType": -4, "signed": {"javaType": "Object"}},
+    {"dbName": "BINARY", "dbType": -2, "signed": {"javaType": "byte[]"}},
+    {"dbName": "VARBINARY", "dbType": -3, "signed": {"javaType": "byte[]"}},
+    {"dbName": "LONGVARBINARY", "dbType": -4, "signed": {"javaType": "byte[]"}},
     {"dbName": "NULL", "dbType": 0, "signed": {"javaType": "Object"}},
     {"dbName": "OTHER", "dbType": 1111, "signed": {"javaType": "Object"}},
     {"dbName": "JAVA_OBJECT", "dbType": 2000, "signed": {"javaType": "Object"}},