You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ak...@apache.org on 2015/08/20 12:44:22 UTC

[1/3] incubator-ignite git commit: IGNITE-1261: Implemented schema selection for schema import utility.

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-1.3.3-p3 2054f197f -> 3acc01471


IGNITE-1261: Implemented schema selection for schema import utility.


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

Branch: refs/heads/ignite-1.3.3-p3
Commit: e0f445eead0a1d14b2fe5a498cb0de57ecd18f1d
Parents: 2673eca
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Thu Aug 20 17:43:40 2015 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Thu Aug 20 17:43:40 2015 +0700

----------------------------------------------------------------------
 .../ignite/schema/model/PojoDescriptor.java     |   6 +-
 .../ignite/schema/model/SchemaDescriptor.java   |  61 +++++++
 .../schema/parser/DatabaseMetadataParser.java   |  54 +++++--
 .../parser/dialect/DB2MetadataDialect.java      |   3 +-
 .../parser/dialect/DatabaseMetadataDialect.java |  13 +-
 .../parser/dialect/JdbcMetadataDialect.java     | 129 ++++++++++-----
 .../parser/dialect/MySQLMetadataDialect.java    |  57 +++++++
 .../parser/dialect/OracleMetadataDialect.java   | 111 +++++++++----
 .../org/apache/ignite/schema/ui/Controls.java   |  25 ++-
 .../ignite/schema/ui/SchemaImportApp.java       | 160 ++++++++++++++++---
 .../schema/test/AbstractSchemaImportTest.java   |  10 +-
 11 files changed, 508 insertions(+), 121 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e0f445ee/modules/schema-import/src/main/java/org/apache/ignite/schema/model/PojoDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/schema-import/src/main/java/org/apache/ignite/schema/model/PojoDescriptor.java b/modules/schema-import/src/main/java/org/apache/ignite/schema/model/PojoDescriptor.java
index 5a4a1fa..d35c934 100644
--- a/modules/schema-import/src/main/java/org/apache/ignite/schema/model/PojoDescriptor.java
+++ b/modules/schema-import/src/main/java/org/apache/ignite/schema/model/PojoDescriptor.java
@@ -493,6 +493,8 @@ public class PojoDescriptor {
             case NCHAR:
             case NVARCHAR:
             case LONGNVARCHAR:
+            case CLOB:
+            case NCLOB:
                 return String.class;
 
             case DATE:
@@ -504,10 +506,6 @@ public class PojoDescriptor {
             case TIMESTAMP:
                 return java.sql.Timestamp.class;
 
-            case CLOB:
-            case NCLOB:
-                return String.class;
-
             // BINARY, VARBINARY, LONGVARBINARY, ARRAY, BLOB, NULL, DATALINK
             // OTHER, JAVA_OBJECT, DISTINCT, STRUCT, REF, ROWID, SQLXML
             default:

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e0f445ee/modules/schema-import/src/main/java/org/apache/ignite/schema/model/SchemaDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/schema-import/src/main/java/org/apache/ignite/schema/model/SchemaDescriptor.java b/modules/schema-import/src/main/java/org/apache/ignite/schema/model/SchemaDescriptor.java
new file mode 100644
index 0000000..0ab3c57
--- /dev/null
+++ b/modules/schema-import/src/main/java/org/apache/ignite/schema/model/SchemaDescriptor.java
@@ -0,0 +1,61 @@
+/*
+ * 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.schema.model;
+
+import javafx.beans.property.*;
+
+/**
+ * Descriptor for schema.
+ */
+public class SchemaDescriptor {
+    /** Schema name */
+    private final String schema;
+
+    /** State of schema selection. */
+    private final BooleanProperty selected;
+
+    /**
+     * Constructor of schema descriptor.
+     *
+     * @param schema Schema.
+     * @param selected Selection state.
+     */
+    public SchemaDescriptor(String schema, boolean selected) {
+        this.schema = schema;
+        this.selected = new SimpleBooleanProperty(selected);
+    }
+
+    /**
+     * @return Schema name.
+     */
+    public String schema() {
+        return schema;
+    }
+
+    /**
+     * @return Boolean property support for {@code selected} property.
+     */
+    public BooleanProperty selected() {
+        return selected;
+    }
+
+    @Override
+    public String toString() {
+        return schema;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e0f445ee/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/DatabaseMetadataParser.java
----------------------------------------------------------------------
diff --git a/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/DatabaseMetadataParser.java b/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/DatabaseMetadataParser.java
index 696ca62..9f4450f 100644
--- a/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/DatabaseMetadataParser.java
+++ b/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/DatabaseMetadataParser.java
@@ -33,37 +33,67 @@ public class DatabaseMetadataParser {
     private static final Logger log = Logger.getLogger(DatabaseMetadataParser.class.getName());
 
     /**
-     * Parse database metadata.
+     * Get specified dialect object for selected database.
      *
      * @param conn Connection to database.
-     * @param tblsOnly If {@code true} then process tables only else process tables and views.
-     * @return Collection of POJO descriptors.
-     * @throws SQLException If parsing failed.
+     * @return Specific dialect object.
      */
-    public static ObservableList<PojoDescriptor> parse(Connection conn, boolean tblsOnly) throws SQLException {
-        DatabaseMetadataDialect dialect;
-
+    private static DatabaseMetadataDialect dialect(Connection conn) {
         try {
             String dbProductName = conn.getMetaData().getDatabaseProductName();
 
             if ("Oracle".equals(dbProductName))
-                dialect = new OracleMetadataDialect();
+                return new OracleMetadataDialect();
             else if (dbProductName.startsWith("DB2/"))
-                dialect = new DB2MetadataDialect();
+                return new DB2MetadataDialect();
+            else if (dbProductName.equals("MySQL"))
+                return new MySQLMetadataDialect();
             else
-                dialect = new JdbcMetadataDialect();
+                return new JdbcMetadataDialect();
         }
         catch (SQLException e) {
             log.log(Level.SEVERE, "Failed to resolve dialect (JdbcMetaDataDialect will be used.", e);
 
-            dialect = new JdbcMetadataDialect();
+            return new JdbcMetadataDialect();
         }
+    }
+
+    /**
+     * Get list of schemas from database.
+     *
+     * @param conn Connection to database.
+     * @return List of schema descriptors.
+     * @throws SQLException If shemas loading failed.
+     */
+    public static ObservableList<SchemaDescriptor> schemas(Connection conn) throws SQLException  {
+        List<String> dbSchemas = dialect(conn).schemas(conn);
+
+        List<SchemaDescriptor> uiSchemas = new ArrayList<>(dbSchemas.size());
+
+        for (String schema : dbSchemas)
+            uiSchemas.add(new SchemaDescriptor(schema, false));
+
+        return FXCollections.observableList(uiSchemas);
+    }
+
+    /**
+     * Parse database metadata.
+     *
+     * @param conn Connection to database.
+     * @param schemas Collention of schema names to load.
+     * @param tblsOnly If {@code true} then process tables only else process tables and views.
+     * @return Collection of POJO descriptors.
+     * @throws SQLException If parsing failed.
+     */
+    public static ObservableList<PojoDescriptor> parse(Connection conn, List<String> schemas, boolean tblsOnly)
+        throws SQLException {
+        DatabaseMetadataDialect dialect = dialect(conn);
 
         Map<String, PojoDescriptor> parents = new HashMap<>();
 
         Map<String, Collection<PojoDescriptor>> childrens = new HashMap<>();
 
-        for (DbTable tbl : dialect.tables(conn, tblsOnly)) {
+        for (DbTable tbl : dialect.tables(conn, schemas, tblsOnly)) {
             String schema = tbl.schema();
 
             PojoDescriptor parent = parents.get(schema);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e0f445ee/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/DB2MetadataDialect.java
----------------------------------------------------------------------
diff --git a/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/DB2MetadataDialect.java b/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/DB2MetadataDialect.java
index 17eb8b2..15063e2 100644
--- a/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/DB2MetadataDialect.java
+++ b/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/DB2MetadataDialect.java
@@ -25,6 +25,7 @@ import java.util.*;
 public class DB2MetadataDialect extends JdbcMetadataDialect {
     /** {@inheritDoc} */
     @Override public Set<String> systemSchemas() {
-        return new HashSet<>(Arrays.asList("SYSIBM", "SYSCAT", "SYSSTAT", "SYSTOOLS"));
+        return new HashSet<>(Arrays.asList("SYSIBM", "SYSCAT", "SYSSTAT", "SYSTOOLS", "SYSFUN", "SYSIBMADM",
+            "SYSIBMINTERNAL", "SYSIBMTS", "SYSPROC", "SYSPUBLIC"));
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e0f445ee/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/DatabaseMetadataDialect.java
----------------------------------------------------------------------
diff --git a/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/DatabaseMetadataDialect.java b/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/DatabaseMetadataDialect.java
index 0d17567..9c059b8 100644
--- a/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/DatabaseMetadataDialect.java
+++ b/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/DatabaseMetadataDialect.java
@@ -27,14 +27,25 @@ import java.util.*;
  */
 public abstract class DatabaseMetadataDialect {
     /**
+     * Gets schemas from database.
+     *
+     * @param conn Database connection.
+     * @return Collection of schema descriptors.
+     * @throws SQLException If failed to get schemas.
+     */
+    public abstract List<String> schemas(Connection conn) throws SQLException;
+
+    /**
      * Gets tables from database.
      *
      * @param conn Database connection.
+     * @param schemas Collention of schema names to load.
      * @param tblsOnly If {@code true} then gets only tables otherwise gets tables and views.
      * @return Collection of table descriptors.
      * @throws SQLException If failed to get tables.
      */
-    public abstract Collection<DbTable> tables(Connection conn, boolean tblsOnly) throws SQLException;
+    public abstract Collection<DbTable> tables(Connection conn, List<String> schemas, boolean tblsOnly)
+        throws SQLException;
 
     /**
      * @return Collection of database system schemas.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e0f445ee/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/JdbcMetadataDialect.java
----------------------------------------------------------------------
diff --git a/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/JdbcMetadataDialect.java b/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/JdbcMetadataDialect.java
index ab65e7a..1bb6840 100644
--- a/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/JdbcMetadataDialect.java
+++ b/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/JdbcMetadataDialect.java
@@ -63,76 +63,117 @@ public class JdbcMetadataDialect extends DatabaseMetadataDialect {
     private static final int IDX_ASC_OR_DESC_IDX = 10;
 
     /** {@inheritDoc} */
-    @Override public Collection<DbTable> tables(Connection conn, boolean tblsOnly) throws SQLException {
+    @Override public List<String> schemas(Connection conn) throws SQLException {
+        List<String> schemas = new ArrayList<>();
+
+        ResultSet rs = conn.getMetaData().getSchemas();
+
+        Set<String> sys = systemSchemas();
+
+        while(rs.next()) {
+            String schema = rs.getString(1);
+
+            // Skip system schemas.
+            if (sys.contains(schema))
+                continue;
+
+            schemas.add(schema);
+        }
+
+        return schemas;
+    }
+
+    /**
+     * @return If {@code true} use catalogs for table division.
+     */
+    protected boolean useCatalog() {
+        return false;
+    }
+
+    /**
+     * @return If {@code true} use schemas for table division.
+     */
+    protected boolean useSchema() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<DbTable> tables(Connection conn, List<String> schemas, boolean tblsOnly)
+        throws SQLException {
         DatabaseMetaData dbMeta = conn.getMetaData();
 
         Set<String> sys = systemSchemas();
 
         Collection<DbTable> tbls = new ArrayList<>();
 
-        try (ResultSet tblsRs = dbMeta.getTables(null, null, "%",
-            tblsOnly ? TABLES_ONLY : TABLES_AND_VIEWS)) {
-            while (tblsRs.next()) {
-                String tblCatalog = tblsRs.getString(TBL_CATALOG_IDX);
-                String tblSchema = tblsRs.getString(TBL_SCHEMA_IDX);
-                String tblName = tblsRs.getString(TBL_NAME_IDX);
+        if (schemas.size() == 0)
+            schemas.add(null);
 
-                // In case of MySql we should use catalog.
-                String schema = tblSchema != null ? tblSchema : tblCatalog;
+        for (String toSchema: schemas) {
+            try (ResultSet tblsRs = dbMeta.getTables(useCatalog() ? toSchema : null, useSchema() ? toSchema : null, "%",
+                    tblsOnly ? TABLES_ONLY : TABLES_AND_VIEWS)) {
+                while (tblsRs.next()) {
+                    String tblCatalog = tblsRs.getString(TBL_CATALOG_IDX);
+                    String tblSchema = tblsRs.getString(TBL_SCHEMA_IDX);
+                    String tblName = tblsRs.getString(TBL_NAME_IDX);
 
-                // Skip system schemas.
-                if (sys.contains(schema))
-                    continue;
+                    // In case of MySql we should use catalog.
+                    String schema = tblSchema != null ? tblSchema : tblCatalog;
 
-                Set<String> pkCols = new HashSet<>();
+                    // Skip system schemas.
+                    if (sys.contains(schema))
+                        continue;
 
-                try (ResultSet pkRs = dbMeta.getPrimaryKeys(tblCatalog, tblSchema, tblName)) {
-                    while (pkRs.next())
-                        pkCols.add(pkRs.getString(PK_COL_NAME_IDX));
-                }
+                    Set<String> pkCols = new HashSet<>();
+
+                    try (ResultSet pkRs = dbMeta.getPrimaryKeys(tblCatalog, tblSchema, tblName)) {
+                        while (pkRs.next())
+                            pkCols.add(pkRs.getString(PK_COL_NAME_IDX));
+                    }
 
-                List<DbColumn> cols = new ArrayList<>();
+                    List<DbColumn> cols = new ArrayList<>();
 
-                try (ResultSet colsRs = dbMeta.getColumns(tblCatalog, tblSchema, tblName, null)) {
-                    while (colsRs.next()) {
-                        String colName = colsRs.getString(COL_NAME_IDX);
+                    try (ResultSet colsRs = dbMeta.getColumns(tblCatalog, tblSchema, tblName, null)) {
+                        while (colsRs.next()) {
+                            String colName = colsRs.getString(COL_NAME_IDX);
 
-                        cols.add(new DbColumn(
-                            colName,
-                            colsRs.getInt(COL_DATA_TYPE_IDX),
-                            pkCols.contains(colName),
-                            colsRs.getInt(COL_NULLABLE_IDX) == DatabaseMetaData.columnNullable));
+                            cols.add(new DbColumn(
+                                    colName,
+                                    colsRs.getInt(COL_DATA_TYPE_IDX),
+                                    pkCols.contains(colName),
+                                    colsRs.getInt(COL_NULLABLE_IDX) == DatabaseMetaData.columnNullable));
+                        }
                     }
-                }
 
-                Map<String, Map<String, Boolean>> idxs = new LinkedHashMap<>();
+                    Map<String, Map<String, Boolean>> idxs = new LinkedHashMap<>();
 
-                try (ResultSet idxRs = dbMeta.getIndexInfo(tblCatalog, tblSchema, tblName, false, true)) {
-                    while (idxRs.next()) {
-                        String idxName = idxRs.getString(IDX_NAME_IDX);
+                    try (ResultSet idxRs = dbMeta.getIndexInfo(tblCatalog, tblSchema, tblName, false, true)) {
+                        while (idxRs.next()) {
+                            String idxName = idxRs.getString(IDX_NAME_IDX);
 
-                        String colName = idxRs.getString(IDX_COL_NAME_IDX);
+                            String colName = idxRs.getString(IDX_COL_NAME_IDX);
 
-                        if (idxName == null || colName == null)
-                            continue;
+                            if (idxName == null || colName == null)
+                                continue;
 
-                        Map<String, Boolean> idx = idxs.get(idxName);
+                            Map<String, Boolean> idx = idxs.get(idxName);
 
-                        if (idx == null) {
-                            idx = new LinkedHashMap<>();
+                            if (idx == null) {
+                                idx = new LinkedHashMap<>();
 
-                            idxs.put(idxName, idx);
-                        }
+                                idxs.put(idxName, idx);
+                            }
 
-                        String askOrDesc = idxRs.getString(IDX_ASC_OR_DESC_IDX);
+                            String askOrDesc = idxRs.getString(IDX_ASC_OR_DESC_IDX);
 
-                        Boolean desc = askOrDesc != null ? "D".equals(askOrDesc) : null;
+                            Boolean desc = askOrDesc != null ? "D".equals(askOrDesc) : null;
 
-                        idx.put(colName, desc);
+                            idx.put(colName, desc);
+                        }
                     }
-                }
 
-                tbls.add(table(schema, tblName, cols, idxs));
+                    tbls.add(table(schema, tblName, cols, idxs));
+                }
             }
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e0f445ee/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/MySQLMetadataDialect.java
----------------------------------------------------------------------
diff --git a/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/MySQLMetadataDialect.java b/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/MySQLMetadataDialect.java
new file mode 100644
index 0000000..b592321
--- /dev/null
+++ b/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/MySQLMetadataDialect.java
@@ -0,0 +1,57 @@
+/*
+ * 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.schema.parser.dialect;
+
+import java.sql.*;
+import java.util.*;
+
+/**
+ * MySQL specific metadata dialect.
+ */
+public class MySQLMetadataDialect extends JdbcMetadataDialect {
+    /** {@inheritDoc} */
+    @Override public List<String> schemas(Connection conn) throws SQLException {
+        List<String> schemas = new ArrayList<>();
+
+        ResultSet rs = conn.getMetaData().getCatalogs();
+
+        Set<String> sys = systemSchemas();
+
+        while(rs.next()) {
+            String schema = rs.getString(1);
+
+            // Skip system schemas.
+            if (sys.contains(schema))
+                continue;
+
+            schemas.add(schema);
+        }
+
+        return schemas;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean useCatalog() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean useSchema() {
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e0f445ee/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/OracleMetadataDialect.java
----------------------------------------------------------------------
diff --git a/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/OracleMetadataDialect.java b/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/OracleMetadataDialect.java
index 30dda5d..c569a29 100644
--- a/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/OracleMetadataDialect.java
+++ b/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/OracleMetadataDialect.java
@@ -31,8 +31,8 @@ public class OracleMetadataDialect extends DatabaseMetadataDialect {
     /** SQL to get columns metadata. */
     private static final String SQL_COLUMNS = "SELECT a.owner, a.table_name, a.column_name, a.nullable," +
         " a.data_type, a.data_precision, a.data_scale " +
-        "FROM all_tab_columns a %s" +
-        " WHERE a.owner = '%s'" +
+        "FROM all_tab_columns a %s " +
+        " %s " +
         " ORDER BY a.owner, a.table_name, a.column_id";
 
     /** SQL to get list of PRIMARY KEYS columns. */
@@ -81,6 +81,36 @@ public class OracleMetadataDialect extends DatabaseMetadataDialect {
     /** Index column sort order index. */
     private static final int IDX_COL_DESCEND_IDX = 4;
 
+    /** {@inheritDoc} */
+    @Override public Set<String> systemSchemas() {
+        return new HashSet<>(Arrays.asList("ANONYMOUS", "CTXSYS", "DBSNMP", "EXFSYS", "LBACSYS", "MDSYS", "MGMT_VIEW",
+            "OLAPSYS", "OWBSYS", "ORDPLUGINS", "ORDSYS", "OUTLN", "SI_INFORMTN_SCHEMA", "SYS", "SYSMAN", "SYSTEM",
+            "TSMSYS", "WK_TEST", "WKSYS", "WKPROXY", "WMSYS", "XDB",
+
+            "APEX_040000", "APEX_PUBLIC_USER", "DIP", "FLOWS_30000", "FLOWS_FILES", "MDDATA", "ORACLE_OCM",
+            "SPATIAL_CSW_ADMIN_USR", "SPATIAL_WFS_ADMIN_USR", "XS$NULL",
+
+            "BI", "HR", "OE", "PM", "IX", "SH"));
+    }
+
+    /** {@inheritDoc} */
+    @Override public List<String> schemas(Connection conn) throws SQLException {
+        List<String> schemas = new ArrayList<>();
+
+        ResultSet rs = conn.getMetaData().getSchemas();
+
+        Set<String> sysSchemas = systemSchemas();
+
+        while(rs.next()) {
+            String schema = rs.getString(1);
+
+            if (!sysSchemas.contains(schema) && !schema.startsWith("FLOWS_"))
+                schemas.add(schema);
+        }
+
+        return schemas;
+    }
+
     /**
      * @param rs Result set with column type metadata from Oracle database.
      * @return JDBC type.
@@ -225,57 +255,70 @@ public class OracleMetadataDialect extends DatabaseMetadataDialect {
     }
 
     /** {@inheritDoc} */
-    @Override public Collection<DbTable> tables(Connection conn, boolean tblsOnly) throws SQLException {
+    @Override public Collection<DbTable> tables(Connection conn, List<String> schemas, boolean tblsOnly)
+        throws SQLException {
         Collection<DbTable> tbls = new ArrayList<>();
 
         PreparedStatement pkStmt = conn.prepareStatement(SQL_PRIMARY_KEYS);
 
         PreparedStatement idxStmt = conn.prepareStatement(SQL_INDEXES);
 
+        if (schemas.size() == 0)
+            schemas.add(null);
+
+        Set<String> sysSchemas = systemSchemas();
+
         try (Statement colsStmt = conn.createStatement()) {
-            Collection<DbColumn> cols = new ArrayList<>();
+            for (String schema: schemas) {
+                if (systemSchemas().contains(schema) || (schema != null && schema.startsWith("FLOWS_")))
+                    continue;
 
-            Set<String> pkCols = Collections.emptySet();
-            Map<String, Map<String, Boolean>> idxs = Collections.emptyMap();
+                Collection<DbColumn> cols = new ArrayList<>();
 
-            String user = conn.getMetaData().getUserName().toUpperCase();
+                Set<String> pkCols = Collections.emptySet();
+                Map<String, Map<String, Boolean>> idxs = Collections.emptyMap();
 
-            String sql = String.format(SQL_COLUMNS,
-                tblsOnly ? "INNER JOIN all_tables b on a.table_name = b.table_name and a.owner = b.owner" : "", user);
+                String sql = String.format(SQL_COLUMNS,
+                        tblsOnly ? "INNER JOIN all_tables b on a.table_name = b.table_name and a.owner = b.owner" : "",
+                        schema != null ? String.format(" WHERE a.owner = '%s' ", schema) : "");
 
-            try (ResultSet colsRs = colsStmt.executeQuery(sql)) {
-                String prevSchema = "";
-                String prevTbl = "";
+                try (ResultSet colsRs = colsStmt.executeQuery(sql)) {
+                    String prevSchema = "";
+                    String prevTbl = "";
 
-                boolean first = true;
+                    boolean first = true;
 
-                while (colsRs.next()) {
-                    String owner = colsRs.getString(OWNER_IDX);
-                    String tbl = colsRs.getString(TBL_NAME_IDX);
+                    while (colsRs.next()) {
+                        String owner = colsRs.getString(OWNER_IDX);
+                        String tbl = colsRs.getString(TBL_NAME_IDX);
 
-                    boolean changed = !owner.equals(prevSchema) || !tbl.equals(prevTbl);
+                        if (sysSchemas.contains(owner) || (schema != null && schema.startsWith("FLOWS_")))
+                            continue;
 
-                    if (changed) {
-                        if (first)
-                            first = false;
-                        else
-                            tbls.add(table(prevSchema, prevTbl, cols, idxs));
+                        boolean changed = !owner.equals(prevSchema) || !tbl.equals(prevTbl);
 
-                        prevSchema = owner;
-                        prevTbl = tbl;
-                        cols = new ArrayList<>();
-                        pkCols = primaryKeys(pkStmt, owner, tbl);
-                        idxs = indexes(idxStmt, owner, tbl);
-                    }
+                        if (changed) {
+                            if (first)
+                                first = false;
+                            else
+                                tbls.add(table(prevSchema, prevTbl, cols, idxs));
 
-                    String colName = colsRs.getString(COL_NAME_IDX);
+                            prevSchema = owner;
+                            prevTbl = tbl;
+                            cols = new ArrayList<>();
+                            pkCols = primaryKeys(pkStmt, owner, tbl);
+                            idxs = indexes(idxStmt, owner, tbl);
+                        }
 
-                    cols.add(new DbColumn(colName, decodeType(colsRs), pkCols.contains(colName),
-                        !"N".equals(colsRs.getString(NULLABLE_IDX))));
-                }
+                        String colName = colsRs.getString(COL_NAME_IDX);
 
-                if (!cols.isEmpty())
-                    tbls.add(table(prevSchema, prevTbl, cols, idxs));
+                        cols.add(new DbColumn(colName, decodeType(colsRs), pkCols.contains(colName),
+                                !"N".equals(colsRs.getString(NULLABLE_IDX))));
+                    }
+
+                    if (!cols.isEmpty())
+                        tbls.add(table(prevSchema, prevTbl, cols, idxs));
+                }
             }
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e0f445ee/modules/schema-import/src/main/java/org/apache/ignite/schema/ui/Controls.java
----------------------------------------------------------------------
diff --git a/modules/schema-import/src/main/java/org/apache/ignite/schema/ui/Controls.java b/modules/schema-import/src/main/java/org/apache/ignite/schema/ui/Controls.java
index 6c4d6bd..794fe6f 100644
--- a/modules/schema-import/src/main/java/org/apache/ignite/schema/ui/Controls.java
+++ b/modules/schema-import/src/main/java/org/apache/ignite/schema/ui/Controls.java
@@ -350,17 +350,40 @@ public class Controls {
      *
      * @param title Title.
      * @param node Node.
+     * @param collapsible Collapsible flag.
      * @return New {@code TitledPane} instance.
      */
-    public static TitledPane titledPane(String title, Node node) {
+    public static TitledPane titledPane(String title, Node node, boolean collapsible) {
         TitledPane tp = new TitledPane(title, node);
 
+        tp.setCollapsible(collapsible);
         tp.setExpanded(false);
 
         return tp;
     }
 
     /**
+     * Create list view.
+     *
+     * @param tip Tooltip text.
+     * @param cb Callback function for list view cell data binding.
+     * @param <T> Type of showed by viewer element.
+     * @return New {@code ListView} instance.
+     */
+    public static <T> ListView<T> list(String tip, Callback<T, ObservableValue<Boolean>> cb) {
+        ListView lst = new ListView<>();
+
+        lst.setCellFactory(CheckBoxListCell.forListView(cb));
+
+        lst.setMinHeight(70);
+        lst.getSelectionModel().setSelectionMode(SelectionMode.MULTIPLE);
+
+        tooltip(lst, tip);
+
+        return lst;
+    }
+
+    /**
      * Create table column.
      *
      * @param colName Column name to display.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e0f445ee/modules/schema-import/src/main/java/org/apache/ignite/schema/ui/SchemaImportApp.java
----------------------------------------------------------------------
diff --git a/modules/schema-import/src/main/java/org/apache/ignite/schema/ui/SchemaImportApp.java b/modules/schema-import/src/main/java/org/apache/ignite/schema/ui/SchemaImportApp.java
index 7b9c220..aab36b8 100644
--- a/modules/schema-import/src/main/java/org/apache/ignite/schema/ui/SchemaImportApp.java
+++ b/modules/schema-import/src/main/java/org/apache/ignite/schema/ui/SchemaImportApp.java
@@ -201,6 +201,9 @@ public class SchemaImportApp extends Application {
     private ComboBox<String> parseCb;
 
     /** */
+    private ListView<SchemaDescriptor> schemaLst;
+
+    /** */
     private GridPaneEx connPnl;
 
     /** */
@@ -245,6 +248,8 @@ public class SchemaImportApp extends Application {
     /** */
     private ProgressIndicator pi;
 
+    private ObservableList<SchemaDescriptor> schemas = FXCollections.emptyObservableList();
+
     /** List with POJOs descriptors. */
     private ObservableList<PojoDescriptor> pojos = FXCollections.emptyObservableList();
 
@@ -318,11 +323,12 @@ public class SchemaImportApp extends Application {
     }
 
     /**
-     * Fill tree with database metadata.
+     * Open connection to database.
+     *
+     * @return Connection to database.
+     * @throws SQLException If connection failed.
      */
-    private void fill() {
-        lockUI(connLayerPnl, connPnl, nextBtn);
-
+    private Connection connect() throws SQLException {
         final String jdbcDrvJarPath = jdbcDrvJarTf.getText().trim();
 
         final String jdbcDrvCls = jdbcDrvClsTf.getText();
@@ -341,6 +347,27 @@ public class SchemaImportApp extends Application {
         if (!pwd.isEmpty())
             jdbcInfo.put("password", pwd);
 
+        return connect(jdbcDrvJarPath, jdbcDrvCls, jdbcUrl, jdbcInfo);
+    }
+
+    /**
+     * Fill tree with database metadata.
+     */
+    private void fill() {
+        final List<String> selSchemas = new ArrayList<>();
+
+        for (SchemaDescriptor schema: schemas)
+            if (schema.selected().getValue())
+                selSchemas.add(schema.schema());
+
+        if (selSchemas.size() == 0)
+            if (!MessageBox.confirmDialog(owner, "No schemas selected.\nExtract tables for all available schemas?"))
+                return;
+
+        lockUI(connLayerPnl, connPnl, nextBtn);
+
+        final String jdbcUrl = jdbcUrlTf.getText();
+
         final boolean tblsOnly = parseCb.getSelectionModel().getSelectedIndex() == 0;
 
         Runnable task = new Task<Void>() {
@@ -348,8 +375,8 @@ public class SchemaImportApp extends Application {
             @Override protected Void call() throws Exception {
                 long started = System.currentTimeMillis();
 
-                try (Connection conn = connect(jdbcDrvJarPath, jdbcDrvCls, jdbcUrl, jdbcInfo)) {
-                    pojos = DatabaseMetadataParser.parse(conn, tblsOnly);
+                try (Connection conn = connect()) {
+                    pojos = DatabaseMetadataParser.parse(conn, selSchemas, tblsOnly);
                 }
 
                 perceptualDelay(started);
@@ -377,7 +404,6 @@ public class SchemaImportApp extends Application {
 
                     pojosTbl.requestFocus();
 
-
                     hdrPane.setLeft(genIcon);
 
                     titleLb.setText("Generate XML And POJOs");
@@ -415,6 +441,69 @@ public class SchemaImportApp extends Application {
     }
 
     /**
+     * Load schemas list from database.
+     */
+    private void loadSchemas() {
+        lockUI(connLayerPnl, connPnl, nextBtn);
+
+        final String jdbcUrl = jdbcUrlTf.getText();
+
+        Runnable task = new Task<Void>() {
+            /** {@inheritDoc} */
+            @Override protected Void call() throws Exception {
+                long started = System.currentTimeMillis();
+
+                try (Connection conn = connect()) {
+                    schemas = DatabaseMetadataParser.schemas(conn);
+                }
+
+                perceptualDelay(started);
+
+                return null;
+            }
+
+            /** {@inheritDoc} */
+            @Override protected void succeeded() {
+                try {
+                    super.succeeded();
+
+                    schemaLst.setItems(schemas);
+
+                    if (schemas.isEmpty()) {
+                        MessageBox.warningDialog(owner, "No schemas found in database. Recheck JDBC URL.\n" +
+                            "JDBC URL: " +  jdbcUrl);
+
+                        return;
+                    }
+
+                    nextBtn.setDisable(false);
+                }
+                finally {
+                    unlockUI(connLayerPnl, connPnl, nextBtn);
+                }
+            }
+
+            /** {@inheritDoc} */
+            @Override protected void cancelled() {
+                super.cancelled();
+
+                unlockUI(connLayerPnl, connPnl, nextBtn);
+            }
+
+            /** {@inheritDoc} */
+            @Override protected void failed() {
+                super.succeeded();
+
+                unlockUI(connLayerPnl, connPnl, nextBtn);
+
+                MessageBox.errorDialog(owner, "Failed to get schemas list from database.", getException());
+            }
+        };
+
+        exec.submit(task);
+    }
+
+    /**
      * Generate XML and POJOs.
      */
     private void generate() {
@@ -426,7 +515,7 @@ public class SchemaImportApp extends Application {
             return;
         }
 
-        if (checkInput(outFolderTf, true, "Output folder should not be empty!"))
+        if (!checkInput(outFolderTf, true, "Output folder should not be empty!"))
             return;
 
         lockUI(genLayerPnl, genPnl, prevBtn, nextBtn);
@@ -629,25 +718,22 @@ public class SchemaImportApp extends Application {
 
             MessageBox.warningDialog(owner, msg);
 
-            return true;
+            return false;
         }
 
-        return false;
+        return true;
     }
 
     /**
      * Go to &quot;Generate XML And POJOs&quot; panel or generate XML and POJOs.
      */
     private void next() {
-        if (rootPane.getCenter() == connLayerPnl) {
-            if (checkInput(jdbcDrvJarTf, true, "Path to JDBC driver is not specified!") ||
-                checkInput(jdbcDrvClsTf, true, "JDBC driver class name is not specified!") ||
-                checkInput(jdbcUrlTf, true, "JDBC URL connection string is not specified!") ||
+        if (rootPane.getCenter() == connLayerPnl)
+            if (checkInput(jdbcDrvJarTf, true, "Path to JDBC driver is not specified!") &&
+                checkInput(jdbcDrvClsTf, true, "JDBC driver class name is not specified!") &&
+                checkInput(jdbcUrlTf, true, "JDBC URL connection string is not specified!") &&
                 checkInput(userTf, true, "User name is not specified!"))
-                return;
-
-            fill();
-        }
+                fill();
         else
             generate();
     }
@@ -709,6 +795,9 @@ public class SchemaImportApp extends Application {
         connPnl.addColumn(100, 100, Double.MAX_VALUE, Priority.ALWAYS);
         connPnl.addColumn(35, 35, 35, Priority.NEVER);
 
+        connPnl.addRows(9);
+        connPnl.addRow(100, 100, Double.MAX_VALUE, Priority.ALWAYS);
+
         connPnl.add(text("This utility is designed to automatically generate configuration XML files and" +
             " POJO classes from database schema information.", 550), 3);
 
@@ -780,6 +869,27 @@ public class SchemaImportApp extends Application {
 
         parseCb = connPnl.addLabeled("Parse:", comboBox("Type of tables to parse", "Tables only", "Tables and Views"), 2);
 
+        GridPaneEx schemaPnl = paneEx(5, 5, 5, 5);
+        schemaPnl.addColumn(100, 100, Double.MAX_VALUE, Priority.ALWAYS);
+        schemaPnl.addColumn();
+
+        schemaLst = schemaPnl.add(list("Select schemas to load", new SchemaCell()));
+
+        schemaPnl.wrap();
+
+        schemaPnl.add(button("Load schemas", "Load schemas for specified database", new EventHandler<ActionEvent>() {
+            @Override
+            public void handle(ActionEvent evt) {
+                loadSchemas();
+            }
+        }));
+
+        TitledPane titledPnl = connPnl.add(titledPane("Schemas", schemaPnl, false), 3);
+
+        titledPnl.setExpanded(true);
+
+        GridPaneEx.setValignment(titledPnl, VPos.TOP);
+
         connLayerPnl = stackPane(connPnl);
 
         return connLayerPnl;
@@ -987,7 +1097,7 @@ public class SchemaImportApp extends Application {
                     " with the given replacement",
                 new EventHandler<ActionEvent>() {
                     @Override public void handle(ActionEvent evt) {
-                        if (checkInput(regexTf, false, "Regular expression should not be empty!"))
+                        if (!checkInput(regexTf, false, "Regular expression should not be empty!"))
                             return;
 
                         String sel = replaceCb.getSelectionModel().getSelectedItem();
@@ -1137,7 +1247,7 @@ public class SchemaImportApp extends Application {
         });
 
         genPnl.add(titledPane("Rename \"Key class name\", \"Value class name\" or  \"Java name\" for selected tables",
-            regexPnl), 3);
+            regexPnl, true), 3);
 
         genLayerPnl = stackPane(genPnl);
     }
@@ -1648,6 +1758,16 @@ public class SchemaImportApp extends Application {
     }
 
     /**
+     * Special list view cell to select loaded schemas.
+     */
+    private static class SchemaCell implements Callback<SchemaDescriptor, ObservableValue<Boolean>> {
+        @Override
+        public ObservableValue<Boolean> call(SchemaDescriptor item) {
+            return item.selected();
+        }
+    }
+
+    /**
      * Special table cell to select schema or table.
      */
     private static class PojoDescriptorCell extends TableCell<PojoDescriptor, Boolean> {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e0f445ee/modules/schema-import/src/test/java/org/apache/ignite/schema/test/AbstractSchemaImportTest.java
----------------------------------------------------------------------
diff --git a/modules/schema-import/src/test/java/org/apache/ignite/schema/test/AbstractSchemaImportTest.java b/modules/schema-import/src/test/java/org/apache/ignite/schema/test/AbstractSchemaImportTest.java
index 0c3ecb1..b9b9c76 100644
--- a/modules/schema-import/src/test/java/org/apache/ignite/schema/test/AbstractSchemaImportTest.java
+++ b/modules/schema-import/src/test/java/org/apache/ignite/schema/test/AbstractSchemaImportTest.java
@@ -19,13 +19,13 @@ package org.apache.ignite.schema.test;
 
 import junit.framework.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.schema.model.PojoDescriptor;
-import org.apache.ignite.schema.parser.DatabaseMetadataParser;
+import org.apache.ignite.schema.model.*;
+import org.apache.ignite.schema.parser.*;
 import org.apache.ignite.schema.ui.*;
 
 import java.io.*;
 import java.sql.*;
-import java.util.List;
+import java.util.*;
 
 import static org.apache.ignite.schema.ui.MessageBox.Result.*;
 
@@ -93,7 +93,9 @@ public abstract class AbstractSchemaImportTest extends TestCase {
 
         U.closeQuiet(stmt);
 
-        pojos = DatabaseMetadataParser.parse(conn, false);
+        List<String> schemas = new ArrayList<>();
+
+        pojos = DatabaseMetadataParser.parse(conn, schemas, false);
 
         U.closeQuiet(conn);
     }



[2/3] incubator-ignite git commit: Review notes.

Posted by ak...@apache.org.
Review notes.


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

Branch: refs/heads/ignite-1.3.3-p3
Commit: d93e1dbde5b22688530dd4deacbb06467f7e2b13
Parents: e0f445e
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Thu Aug 20 17:44:10 2015 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Thu Aug 20 17:44:10 2015 +0700

----------------------------------------------------------------------
 .../cache/store/jdbc/CacheJdbcPojoStore.java    | 20 ++++++++--------
 .../store/jdbc/CacheJdbcPojoStoreTest.java      | 24 +++++++++++++++-----
 .../jdbc/CacheJdbcPojoStoreFactorySelfTest.java |  3 ---
 3 files changed, 28 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d93e1dbd/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStore.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStore.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStore.java
index 428485f..1ff170e 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStore.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStore.java
@@ -99,8 +99,8 @@ public class CacheJdbcPojoStore<K, V> extends CacheAbstractJdbcStore<K, V> {
                         getters.put(field.getJavaName(), cls.getMethod("is" + prop));
                     }
                     catch (NoSuchMethodException e) {
-                        throw new CacheException("Failed to find getter in POJO class [class name=" + clsName +
-                            ", property=" + field.getJavaName() + "]", e);
+                        throw new CacheException("Failed to find getter in POJO class [clsName=" + clsName +
+                            ", prop=" + field.getJavaName() + "]", e);
                     }
                 }
 
@@ -108,8 +108,8 @@ public class CacheJdbcPojoStore<K, V> extends CacheAbstractJdbcStore<K, V> {
                     setters.put(field.getJavaName(), cls.getMethod("set" + prop, field.getJavaType()));
                 }
                 catch (NoSuchMethodException e) {
-                    throw new CacheException("Failed to find setter in POJO class [class name=" + clsName +
-                        ", property=" + field.getJavaName() + "]", e);
+                    throw new CacheException("Failed to find setter in POJO class [clsName=" + clsName +
+                        ", prop=" + field.getJavaName() + "]", e);
                 }
             }
         }
@@ -172,8 +172,8 @@ public class CacheJdbcPojoStore<K, V> extends CacheAbstractJdbcStore<K, V> {
                 Method setter = mc.setters.get(fldJavaName);
 
                 if (setter == null)
-                    throw new IllegalStateException("Failed to find setter in POJO class [class name=" + typeName +
-                        ", property=" + fldJavaName + "]");
+                    throw new IllegalStateException("Failed to find setter in POJO class [clsName=" + typeName +
+                        ", prop=" + fldJavaName + "]");
 
                 String fldDbName = field.getDatabaseName();
 
@@ -183,8 +183,8 @@ public class CacheJdbcPojoStore<K, V> extends CacheAbstractJdbcStore<K, V> {
                     setter.invoke(obj, getColumnValue(rs, colIdx, field.getJavaType()));
                 }
                 catch (Exception e) {
-                    throw new IllegalStateException("Failed to set property in POJO class [class name=" + typeName +
-                        ", property=" + fldJavaName + ", column=" + colIdx + ", db name=" + fldDbName + "]", e);
+                    throw new IllegalStateException("Failed to set property in POJO class [clsName=" + typeName +
+                        ", prop=" + fldJavaName + ", col=" + colIdx + ", dbName=" + fldDbName + "]", e);
                 }
             }
 
@@ -214,8 +214,8 @@ public class CacheJdbcPojoStore<K, V> extends CacheAbstractJdbcStore<K, V> {
             Method getter = mc.getters.get(fieldName);
 
             if (getter == null)
-                throw new CacheLoaderException("Failed to find getter in POJO class [class name=" + typeName +
-                    ", property=" + fieldName + "]");
+                throw new CacheLoaderException("Failed to find getter in POJO class [clsName=" + typeName +
+                    ", prop=" + fieldName + "]");
 
             return getter.invoke(obj);
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d93e1dbd/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 b1efb0d..68a77dc 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
@@ -183,12 +183,24 @@ public class CacheJdbcPojoStoreTest extends GridAbstractCacheStoreSelfTest<Cache
             // No-op.
         }
 
-        stmt.executeUpdate("CREATE TABLE IF NOT EXISTS String_Entries (key varchar(100) not null, val varchar(100), PRIMARY KEY(key))");
-        stmt.executeUpdate("CREATE TABLE IF NOT EXISTS UUID_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 Organization (id integer not null, name varchar(50), city varchar(50), PRIMARY KEY(id))");
-        stmt.executeUpdate("CREATE TABLE IF NOT EXISTS Person (id integer not null, org_id integer, name varchar(50), PRIMARY KEY(id))");
-        stmt.executeUpdate("CREATE TABLE IF NOT EXISTS Person_Complex (id integer not null, org_id integer not null, city_id integer not null, name varchar(50), salary integer, PRIMARY KEY(id))");
+        stmt.executeUpdate("CREATE TABLE IF NOT EXISTS " +
+            "String_Entries (key varchar(100) not null, val varchar(100), PRIMARY KEY(key))");
+
+        stmt.executeUpdate("CREATE TABLE IF NOT EXISTS " +
+            "UUID_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 " +
+            "Organization (id integer not null, name varchar(50), city varchar(50), PRIMARY KEY(id))");
+
+        stmt.executeUpdate("CREATE TABLE IF NOT EXISTS " +
+            "Person (id integer not null, org_id integer, name varchar(50), PRIMARY KEY(id))");
+
+        stmt.executeUpdate("CREATE TABLE IF NOT EXISTS " +
+            "Person_Complex (id integer not null, org_id integer not null, city_id integer not null, " +
+            "name varchar(50), salary integer, PRIMARY KEY(id))");
 
         conn.commit();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d93e1dbd/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactorySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactorySelfTest.java b/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactorySelfTest.java
index faaf28f..8b4f3f9 100644
--- a/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactorySelfTest.java
+++ b/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactorySelfTest.java
@@ -62,9 +62,6 @@ public class CacheJdbcPojoStoreFactorySelfTest extends GridCommonAbstractTest {
                 checkStore(cache, JdbcDataSource.class);
             }
         }
-        catch (Exception e) {
-            fail("Failed to validate cache configuration. Cache store factory is not serializable.");
-        }
     }
 
     /**


[3/3] incubator-ignite git commit: Merge remote-tracking branch 'origin/ignite-1.3.3-p3' into ignite-1.3.3-p3

Posted by ak...@apache.org.
Merge remote-tracking branch 'origin/ignite-1.3.3-p3' into ignite-1.3.3-p3


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

Branch: refs/heads/ignite-1.3.3-p3
Commit: 3acc01471c613b7b6de4c4c2b20980db2d589c52
Parents: d93e1db 2054f19
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Thu Aug 20 17:44:39 2015 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Thu Aug 20 17:44:39 2015 +0700

----------------------------------------------------------------------
 .../cache/CrossCacheTxRandomOperationsTest.java | 490 +++++++++++++++++++
 ...gniteCachePutRetryTransactionalSelfTest.java |  15 +-
 2 files changed, 500 insertions(+), 5 deletions(-)
----------------------------------------------------------------------