You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by se...@apache.org on 2015/07/28 14:38:23 UTC

incubator-ignite git commit: IGNITE-1155 Split 'ignite-schema-import' to 'ignite-schema-import' and 'ignite-schema-import-db'

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-1155_1 [created] 1e3c77f65


IGNITE-1155 Split 'ignite-schema-import' to 'ignite-schema-import' and 'ignite-schema-import-db'


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

Branch: refs/heads/ignite-1155_1
Commit: 1e3c77f655ab6bb2b1cfd6c8669a9bc18d2d075e
Parents: b9d650a
Author: sevdokimov <se...@gridgain.com>
Authored: Tue Jul 28 15:38:10 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Tue Jul 28 15:38:10 2015 +0300

----------------------------------------------------------------------
 modules/schema-import-db/README.txt             |   4 +
 modules/schema-import-db/pom.xml                |  44 +++
 .../apache/ignite/schema/parser/DBReader.java   |  68 +++++
 .../apache/ignite/schema/parser/DbColumn.java   |  76 +++++
 .../apache/ignite/schema/parser/DbTable.java    | 105 +++++++
 .../parser/dialect/DB2MetadataDialect.java      |  30 ++
 .../parser/dialect/DatabaseMetadataDialect.java |  78 +++++
 .../parser/dialect/JdbcMetadataDialect.java     | 141 ++++++++++
 .../parser/dialect/OracleMetadataDialect.java   | 281 +++++++++++++++++++
 modules/schema-import/pom.xml                   |   6 +
 .../schema/parser/DatabaseMetadataParser.java   |  21 +-
 .../apache/ignite/schema/parser/DbColumn.java   |  76 -----
 .../apache/ignite/schema/parser/DbTable.java    | 105 -------
 .../parser/dialect/DB2MetadataDialect.java      |  30 --
 .../parser/dialect/DatabaseMetadataDialect.java |  78 -----
 .../parser/dialect/JdbcMetadataDialect.java     | 141 ----------
 .../parser/dialect/OracleMetadataDialect.java   | 281 -------------------
 17 files changed, 834 insertions(+), 731 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1e3c77f6/modules/schema-import-db/README.txt
----------------------------------------------------------------------
diff --git a/modules/schema-import-db/README.txt b/modules/schema-import-db/README.txt
new file mode 100644
index 0000000..556e1c6
--- /dev/null
+++ b/modules/schema-import-db/README.txt
@@ -0,0 +1,4 @@
+Apache Ignite Schema Import DB Module
+------------------------------------------
+
+Utility classes to extract database metadata.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1e3c77f6/modules/schema-import-db/pom.xml
----------------------------------------------------------------------
diff --git a/modules/schema-import-db/pom.xml b/modules/schema-import-db/pom.xml
new file mode 100644
index 0000000..c0bfc8d
--- /dev/null
+++ b/modules/schema-import-db/pom.xml
@@ -0,0 +1,44 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  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.
+-->
+
+<!--
+    POM file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.ignite</groupId>
+        <artifactId>ignite-parent</artifactId>
+        <version>1</version>
+        <relativePath>../../parent</relativePath>
+    </parent>
+
+    <artifactId>ignite-schema-import-db</artifactId>
+    <version>1.4.1-SNAPSHOT</version>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-core</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+    </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1e3c77f6/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/DBReader.java
----------------------------------------------------------------------
diff --git a/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/DBReader.java b/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/DBReader.java
new file mode 100644
index 0000000..5c4466e
--- /dev/null
+++ b/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/DBReader.java
@@ -0,0 +1,68 @@
+/*
+ * 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;
+
+import org.apache.ignite.schema.parser.dialect.*;
+
+import java.sql.*;
+import java.util.*;
+import java.util.logging.*;
+
+/**
+ *
+ */
+public class DBReader {
+    /** Logger. */
+    private static final Logger log = Logger.getLogger(DBReader.class.getName());
+
+    /**
+     * Default constructor.
+     */
+    private DBReader() {
+        // No-op.
+    }
+
+    /**
+     * Extract DB metadata.
+     *
+     * @param conn Connection.
+     * @param tblsOnly Tables only flag.
+     */
+    public static Collection<DbTable> extractMetadata(Connection conn, boolean tblsOnly) throws SQLException {
+        DatabaseMetadataDialect dialect;
+
+        try {
+            String dbProductName = conn.getMetaData().getDatabaseProductName();
+
+            if ("Oracle".equals(dbProductName))
+                dialect = new OracleMetadataDialect();
+            else if (dbProductName.startsWith("DB2/"))
+                dialect = new DB2MetadataDialect();
+            else
+                dialect = new JdbcMetadataDialect();
+        }
+        catch (SQLException e) {
+            log.log(Level.SEVERE, "Failed to resolve dialect (JdbcMetaDataDialect will be used.", e);
+
+            dialect = new JdbcMetadataDialect();
+        }
+
+        return dialect.tables(conn, tblsOnly);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1e3c77f6/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/DbColumn.java
----------------------------------------------------------------------
diff --git a/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/DbColumn.java b/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/DbColumn.java
new file mode 100644
index 0000000..8b0c813
--- /dev/null
+++ b/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/DbColumn.java
@@ -0,0 +1,76 @@
+/*
+ * 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;
+
+/**
+ * Database table column.
+ */
+public class DbColumn {
+    /** Column name. */
+    private final String name;
+
+    /** Column JDBC type. */
+    private final int type;
+
+    /** Is this column belongs to primary key. */
+    private final boolean key;
+
+    /** Is {@code NULL} allowed for column in database. */
+    private final boolean nullable;
+
+    /**
+     * @param name Column name.
+     * @param type Column JDBC type.
+     * @param key {@code true} if this column belongs to primary key.
+     * @param nullable {@code true} if {@code NULL } allowed for column in database.
+     */
+    public DbColumn(String name, int type, boolean key, boolean nullable) {
+        this.name = name;
+        this.type = type;
+        this.key = key;
+        this.nullable = nullable;
+    }
+
+    /**
+     * @return Column name.
+     */
+    public String name() {
+        return name;
+    }
+
+    /**
+     * @return Column JDBC type.
+     */
+    public int type() {
+        return type;
+    }
+
+    /**
+     * @return {@code true} if this column belongs to primary key.
+     */
+    public boolean key() {
+        return key;
+    }
+
+    /**
+     * @return nullable {@code true} if {@code NULL } allowed for column in database.
+     */
+    public boolean nullable() {
+        return nullable;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1e3c77f6/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/DbTable.java
----------------------------------------------------------------------
diff --git a/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/DbTable.java b/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/DbTable.java
new file mode 100644
index 0000000..35c7d91
--- /dev/null
+++ b/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/DbTable.java
@@ -0,0 +1,105 @@
+/*
+ * 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;
+
+import java.util.*;
+
+/**
+ * Database table.
+ */
+public class DbTable {
+    /** Schema name. */
+    private final String schema;
+
+    /** Table name. */
+    private final String tbl;
+
+    /** Columns. */
+    private final Collection<DbColumn> cols;
+
+    /** Columns in ascending order. */
+    private final Set<String> ascCols;
+
+    /** Columns in descending order. */
+    private final Set<String> descCols;
+
+    /** Indexes. */
+    private final Map<String, Map<String, Boolean>> idxs;
+
+    /**
+     * Default columns.
+     *
+     * @param schema Schema name.
+     * @param tbl Table name.
+     * @param cols Columns.
+     * @param ascCols Columns in ascending order.
+     * @param descCols Columns in descending order.
+     * @param idxs Indexes;
+     */
+    public DbTable(String schema, String tbl, Collection<DbColumn> cols, Set<String> ascCols, Set<String> descCols,
+        Map<String, Map<String, Boolean>> idxs) {
+        this.schema = schema;
+        this.tbl = tbl;
+        this.cols = cols;
+        this.ascCols = ascCols;
+        this.descCols = descCols;
+        this.idxs = idxs;
+    }
+
+    /**
+     * @return Schema name.
+     */
+    public String schema() {
+        return schema;
+    }
+
+    /**
+     * @return Table name.
+     */
+    public String table() {
+        return tbl;
+    }
+
+    /**
+     * @return Columns.
+     */
+    public Collection<DbColumn> columns() {
+        return cols;
+    }
+
+    /**
+     * @return Fields in ascending order
+     */
+    public Set<String> ascendingColumns() {
+        return ascCols;
+    }
+
+    /**
+     * @return Fields in descending order
+     */
+    public Set<String> descendingColumns() {
+        return descCols;
+    }
+
+    /**
+     * @return Indexes.
+     */
+    public Map<String, Map<String, Boolean>> indexes() {
+        return idxs;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1e3c77f6/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/dialect/DB2MetadataDialect.java
----------------------------------------------------------------------
diff --git a/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/dialect/DB2MetadataDialect.java b/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/dialect/DB2MetadataDialect.java
new file mode 100644
index 0000000..17eb8b2
--- /dev/null
+++ b/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/dialect/DB2MetadataDialect.java
@@ -0,0 +1,30 @@
+/*
+ * 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.util.*;
+
+/**
+ * DB2 specific metadata dialect.
+ */
+public class DB2MetadataDialect extends JdbcMetadataDialect {
+    /** {@inheritDoc} */
+    @Override public Set<String> systemSchemas() {
+        return new HashSet<>(Arrays.asList("SYSIBM", "SYSCAT", "SYSSTAT", "SYSTOOLS"));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1e3c77f6/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/dialect/DatabaseMetadataDialect.java
----------------------------------------------------------------------
diff --git a/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/dialect/DatabaseMetadataDialect.java b/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/dialect/DatabaseMetadataDialect.java
new file mode 100644
index 0000000..0d17567
--- /dev/null
+++ b/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/dialect/DatabaseMetadataDialect.java
@@ -0,0 +1,78 @@
+/*
+ * 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 org.apache.ignite.schema.parser.*;
+
+import java.sql.*;
+import java.util.*;
+
+/**
+ * Base class for database metadata dialect.
+ */
+public abstract class DatabaseMetadataDialect {
+    /**
+     * Gets tables from database.
+     *
+     * @param conn Database connection.
+     * @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;
+
+    /**
+     * @return Collection of database system schemas.
+     */
+    public Set<String> systemSchemas() {
+        return Collections.singleton("INFORMATION_SCHEMA");
+    }
+
+    /**
+     * Create table descriptor.
+     *
+     * @param schema Schema name.
+     * @param tbl Table name.
+     * @param cols Table columns.
+     * @param idxs Table indexes.
+     * @return New {@code DbTable} instance.
+     */
+    protected DbTable table(String schema, String tbl, Collection<DbColumn> cols, Map<String, Map<String, Boolean>>idxs) {
+        Set<String> ascCols = new HashSet<>();
+
+        Set<String> descCols = new HashSet<>();
+
+        for (Map<String, Boolean> idx : idxs.values()) {
+            if (idx.size() == 1)
+                for (Map.Entry<String, Boolean> idxCol : idx.entrySet()) {
+                    String colName = idxCol.getKey();
+
+                    Boolean desc = idxCol.getValue();
+
+                    if (desc != null) {
+                        if (desc)
+                            descCols.add(colName);
+                        else
+                            ascCols.add(colName);
+                    }
+                }
+        }
+
+        return new DbTable(schema, tbl, cols, ascCols, descCols, idxs);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1e3c77f6/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/dialect/JdbcMetadataDialect.java
----------------------------------------------------------------------
diff --git a/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/dialect/JdbcMetadataDialect.java b/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/dialect/JdbcMetadataDialect.java
new file mode 100644
index 0000000..ab65e7a
--- /dev/null
+++ b/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/dialect/JdbcMetadataDialect.java
@@ -0,0 +1,141 @@
+/*
+ * 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 org.apache.ignite.schema.parser.*;
+
+import java.sql.*;
+import java.util.*;
+
+/**
+ * Metadata dialect that uses standard JDBC for reading metadata.
+ */
+public class JdbcMetadataDialect extends DatabaseMetadataDialect {
+    /** */
+    private static final String[] TABLES_ONLY = {"TABLE"};
+
+    /** */
+    private static final String[] TABLES_AND_VIEWS = {"TABLE", "VIEW"};
+
+    /** Schema catalog index. */
+    private static final int TBL_CATALOG_IDX = 1;
+
+    /** Schema name index. */
+    private static final int TBL_SCHEMA_IDX = 2;
+
+    /** Table name index. */
+    private static final int TBL_NAME_IDX = 3;
+
+    /** Primary key column name index. */
+    private static final int PK_COL_NAME_IDX = 4;
+
+    /** Column name index. */
+    private static final int COL_NAME_IDX = 4;
+
+    /** Column data type index. */
+    private static final int COL_DATA_TYPE_IDX = 5;
+
+    /** Column nullable index. */
+    private static final int COL_NULLABLE_IDX = 11;
+
+    /** Index name index. */
+    private static final int IDX_NAME_IDX = 6;
+
+    /** Index column name index. */
+    private static final int IDX_COL_NAME_IDX = 9;
+
+    /** Index column descend index. */
+    private static final int IDX_ASC_OR_DESC_IDX = 10;
+
+    /** {@inheritDoc} */
+    @Override public Collection<DbTable> tables(Connection conn, 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);
+
+                // In case of MySql we should use catalog.
+                String schema = tblSchema != null ? tblSchema : tblCatalog;
+
+                // Skip system schemas.
+                if (sys.contains(schema))
+                    continue;
+
+                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<>();
+
+                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));
+                    }
+                }
+
+                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);
+
+                        String colName = idxRs.getString(IDX_COL_NAME_IDX);
+
+                        if (idxName == null || colName == null)
+                            continue;
+
+                        Map<String, Boolean> idx = idxs.get(idxName);
+
+                        if (idx == null) {
+                            idx = new LinkedHashMap<>();
+
+                            idxs.put(idxName, idx);
+                        }
+
+                        String askOrDesc = idxRs.getString(IDX_ASC_OR_DESC_IDX);
+
+                        Boolean desc = askOrDesc != null ? "D".equals(askOrDesc) : null;
+
+                        idx.put(colName, desc);
+                    }
+                }
+
+                tbls.add(table(schema, tblName, cols, idxs));
+            }
+        }
+
+        return tbls;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1e3c77f6/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/dialect/OracleMetadataDialect.java
----------------------------------------------------------------------
diff --git a/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/dialect/OracleMetadataDialect.java b/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/dialect/OracleMetadataDialect.java
new file mode 100644
index 0000000..855c9f7
--- /dev/null
+++ b/modules/schema-import-db/src/main/java/org/apache/ignite/schema/parser/dialect/OracleMetadataDialect.java
@@ -0,0 +1,281 @@
+/*
+ * 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 org.apache.ignite.schema.parser.*;
+
+import java.sql.*;
+import java.util.*;
+
+import static java.sql.Types.*;
+
+/**
+ * Oracle specific metadata dialect.
+ */
+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'" +
+        " ORDER BY a.owner, a.table_name, a.column_id";
+
+    /** SQL to get list of PRIMARY KEYS columns. */
+    private static final String SQL_PRIMARY_KEYS = "SELECT b.column_name" +
+        " FROM all_constraints a" +
+        "  INNER JOIN all_cons_columns b ON a.owner = b.owner AND a.constraint_name = b.constraint_name" +
+        " WHERE a.owner = ? and a.table_name = ? AND a.constraint_type = 'P'";
+
+    /** SQL to get indexes metadata. */
+    private static final String SQL_INDEXES = "select i.index_name, u.column_expression, i.column_name, i.descend" +
+        " FROM all_ind_columns i" +
+        " LEFT JOIN user_ind_expressions u on u.index_name = i.index_name and i.table_name = u.table_name" +
+        " WHERE i.index_owner = ? and i.table_name = ?" +
+        " ORDER BY i.index_name, i.column_position";
+
+    /** Owner index. */
+    private static final int OWNER_IDX = 1;
+
+    /** Table name index. */
+    private static final int TBL_NAME_IDX = 2;
+
+    /** Column name index. */
+    private static final int COL_NAME_IDX = 3;
+
+    /** Nullable index. */
+    private static final int NULLABLE_IDX = 4;
+
+    /** Data type index. */
+    private static final int DATA_TYPE_IDX = 5;
+
+    /** Numeric precision index. */
+    private static final int DATA_PRECISION_IDX = 6;
+
+    /** Numeric scale index. */
+    private static final int DATA_SCALE_IDX = 7;
+
+    /** Index name index. */
+    private static final int IDX_NAME_IDX = 1;
+
+    /** Index name index. */
+    private static final int IDX_EXPR_IDX = 2;
+
+    /** Index column name index. */
+    private static final int IDX_COL_NAME_IDX = 3;
+
+    /** Index column sort order index. */
+    private static final int IDX_COL_DESCEND_IDX = 4;
+
+    /**
+     * @param rs Result set with column type metadata from Oracle database.
+     * @return JDBC type.
+     * @throws SQLException If failed to decode type.
+     */
+    private int decodeType(ResultSet rs) throws SQLException {
+        switch (rs.getString(DATA_TYPE_IDX)) {
+            case "CHAR":
+            case "NCHAR":
+                return CHAR;
+
+            case "VARCHAR2":
+            case "NVARCHAR2":
+                return VARCHAR;
+
+            case "LONG":
+                return LONGVARCHAR;
+
+            case "LONG RAW":
+                return LONGVARBINARY;
+
+            case "FLOAT":
+                return FLOAT;
+
+            case "NUMBER":
+                int precision = rs.getInt(DATA_PRECISION_IDX);
+                int scale = rs.getInt(DATA_SCALE_IDX);
+
+                if (scale > 0) {
+                    if (scale < 4 && precision < 19)
+                        return FLOAT;
+
+                    if (scale > 4 || precision > 19)
+                        return DOUBLE;
+
+                    return NUMERIC;
+                }
+                else {
+                    if (precision < 1)
+                        return INTEGER;
+
+                    if (precision < 2)
+                        return BOOLEAN;
+
+                    if (precision < 4)
+                        return TINYINT;
+
+                    if (precision < 6)
+                        return SMALLINT;
+
+                    if (precision < 11)
+                        return INTEGER;
+
+                    if (precision < 20)
+                        return BIGINT;
+
+                    return NUMERIC;
+                }
+
+            case "DATE":
+                return DATE;
+
+            case "TIMESTAMP":
+                return TIMESTAMP;
+
+            case "BFILE":
+            case "BLOB":
+                return BLOB;
+
+            case "CLOB":
+            case "NCLOB":
+            case "XMLTYPE":
+                return CLOB;
+        }
+
+        return OTHER;
+    }
+
+    /**
+     * Retrieve primary key columns.
+     *
+     * @param stmt Prepared SQL statement to execute.
+     * @param owner DB owner.
+     * @param tbl Table name.
+     * @return Primary key columns.
+     * @throws SQLException If failed to retrieve primary key columns.
+     */
+    private Set<String> primaryKeys(PreparedStatement stmt, String owner, String tbl) throws SQLException {
+        Set<String> pkCols = new HashSet<>();
+
+        stmt.setString(1, owner);
+        stmt.setString(2, tbl);
+
+        try (ResultSet pkRs = stmt.executeQuery()) {
+            while(pkRs.next())
+                pkCols.add(pkRs.getString(1));
+        }
+
+        return pkCols;
+    }
+
+    /**
+     * Retrieve index columns.
+     *
+     * @param stmt Prepared SQL statement to execute.
+     * @param owner DB owner.
+     * @param tbl Table name.
+     * @return Index columns.
+     * @throws SQLException If failed to retrieve indexes columns.
+     */
+    private Map<String, Map<String, Boolean>> indexes(PreparedStatement stmt, String owner, String tbl)
+        throws SQLException {
+        Map<String, Map<String, Boolean>> idxs = new LinkedHashMap<>();
+
+        stmt.setString(1, owner);
+        stmt.setString(2, tbl);
+
+        try (ResultSet idxsRs = stmt.executeQuery()) {
+            while (idxsRs.next()) {
+                String idxName = idxsRs.getString(IDX_NAME_IDX);
+
+                Map<String, Boolean> idx = idxs.get(idxName);
+
+                if (idx == null) {
+                    idx = new LinkedHashMap<>();
+
+                    idxs.put(idxName, idx);
+                }
+
+                String expr = idxsRs.getString(IDX_EXPR_IDX);
+
+                String col = expr == null ? idxsRs.getString(IDX_COL_NAME_IDX) : expr.replaceAll("\"", "");
+
+                idx.put(col, "DESC".equals(idxsRs.getString(IDX_COL_DESCEND_IDX)));
+            }
+        }
+
+        return idxs;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<DbTable> tables(Connection conn, boolean tblsOnly) throws SQLException {
+        Collection<DbTable> tbls = new ArrayList<>();
+
+        PreparedStatement pkStmt = conn.prepareStatement(SQL_PRIMARY_KEYS);
+
+        PreparedStatement idxStmt = conn.prepareStatement(SQL_INDEXES);
+
+        try (Statement colsStmt = conn.createStatement()) {
+            Collection<DbColumn> cols = new ArrayList<>();
+
+            Set<String> pkCols = Collections.emptySet();
+            Map<String, Map<String, Boolean>> idxs = Collections.emptyMap();
+
+            String user = conn.getMetaData().getUserName().toUpperCase();
+
+            String sql = String.format(SQL_COLUMNS,
+                tblsOnly ? "INNER JOIN all_tables b on a.table_name = b.table_name" : "", user);
+
+            try (ResultSet colsRs = colsStmt.executeQuery(sql)) {
+                String prevSchema = "";
+                String prevTbl = "";
+
+                boolean first = true;
+
+                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 (changed) {
+                        if (first)
+                            first = false;
+                        else
+                            tbls.add(table(prevSchema, prevTbl, cols, idxs));
+
+                        prevSchema = owner;
+                        prevTbl = tbl;
+                        cols = new ArrayList<>();
+                        pkCols = primaryKeys(pkStmt, owner, tbl);
+                        idxs = indexes(idxStmt, owner, tbl);
+                    }
+
+                    String colName = colsRs.getString(COL_NAME_IDX);
+
+                    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));
+            }
+        }
+
+        return tbls;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1e3c77f6/modules/schema-import/pom.xml
----------------------------------------------------------------------
diff --git a/modules/schema-import/pom.xml b/modules/schema-import/pom.xml
index 7364e1e..b4c29e0 100644
--- a/modules/schema-import/pom.xml
+++ b/modules/schema-import/pom.xml
@@ -41,6 +41,12 @@
         </dependency>
 
         <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-schema-import-db</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
+        <dependency>
             <groupId>com.h2database</groupId>
             <artifactId>h2</artifactId>
             <version>1.3.175</version>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1e3c77f6/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..54dc3ee 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
@@ -19,7 +19,6 @@ package org.apache.ignite.schema.parser;
 
 import javafx.collections.*;
 import org.apache.ignite.schema.model.*;
-import org.apache.ignite.schema.parser.dialect.*;
 
 import java.sql.*;
 import java.util.*;
@@ -41,29 +40,11 @@ public class DatabaseMetadataParser {
      * @throws SQLException If parsing failed.
      */
     public static ObservableList<PojoDescriptor> parse(Connection conn, boolean tblsOnly) throws SQLException {
-        DatabaseMetadataDialect dialect;
-
-        try {
-            String dbProductName = conn.getMetaData().getDatabaseProductName();
-
-            if ("Oracle".equals(dbProductName))
-                dialect = new OracleMetadataDialect();
-            else if (dbProductName.startsWith("DB2/"))
-                dialect = new DB2MetadataDialect();
-            else
-                dialect = new JdbcMetadataDialect();
-        }
-        catch (SQLException e) {
-            log.log(Level.SEVERE, "Failed to resolve dialect (JdbcMetaDataDialect will be used.", e);
-
-            dialect = new JdbcMetadataDialect();
-        }
-
         Map<String, PojoDescriptor> parents = new HashMap<>();
 
         Map<String, Collection<PojoDescriptor>> childrens = new HashMap<>();
 
-        for (DbTable tbl : dialect.tables(conn, tblsOnly)) {
+        for (DbTable tbl : DBReader.extractMetadata(conn, tblsOnly)) {
             String schema = tbl.schema();
 
             PojoDescriptor parent = parents.get(schema);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1e3c77f6/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/DbColumn.java
----------------------------------------------------------------------
diff --git a/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/DbColumn.java b/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/DbColumn.java
deleted file mode 100644
index 8b0c813..0000000
--- a/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/DbColumn.java
+++ /dev/null
@@ -1,76 +0,0 @@
-/*
- * 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;
-
-/**
- * Database table column.
- */
-public class DbColumn {
-    /** Column name. */
-    private final String name;
-
-    /** Column JDBC type. */
-    private final int type;
-
-    /** Is this column belongs to primary key. */
-    private final boolean key;
-
-    /** Is {@code NULL} allowed for column in database. */
-    private final boolean nullable;
-
-    /**
-     * @param name Column name.
-     * @param type Column JDBC type.
-     * @param key {@code true} if this column belongs to primary key.
-     * @param nullable {@code true} if {@code NULL } allowed for column in database.
-     */
-    public DbColumn(String name, int type, boolean key, boolean nullable) {
-        this.name = name;
-        this.type = type;
-        this.key = key;
-        this.nullable = nullable;
-    }
-
-    /**
-     * @return Column name.
-     */
-    public String name() {
-        return name;
-    }
-
-    /**
-     * @return Column JDBC type.
-     */
-    public int type() {
-        return type;
-    }
-
-    /**
-     * @return {@code true} if this column belongs to primary key.
-     */
-    public boolean key() {
-        return key;
-    }
-
-    /**
-     * @return nullable {@code true} if {@code NULL } allowed for column in database.
-     */
-    public boolean nullable() {
-        return nullable;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1e3c77f6/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/DbTable.java
----------------------------------------------------------------------
diff --git a/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/DbTable.java b/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/DbTable.java
deleted file mode 100644
index 35c7d91..0000000
--- a/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/DbTable.java
+++ /dev/null
@@ -1,105 +0,0 @@
-/*
- * 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;
-
-import java.util.*;
-
-/**
- * Database table.
- */
-public class DbTable {
-    /** Schema name. */
-    private final String schema;
-
-    /** Table name. */
-    private final String tbl;
-
-    /** Columns. */
-    private final Collection<DbColumn> cols;
-
-    /** Columns in ascending order. */
-    private final Set<String> ascCols;
-
-    /** Columns in descending order. */
-    private final Set<String> descCols;
-
-    /** Indexes. */
-    private final Map<String, Map<String, Boolean>> idxs;
-
-    /**
-     * Default columns.
-     *
-     * @param schema Schema name.
-     * @param tbl Table name.
-     * @param cols Columns.
-     * @param ascCols Columns in ascending order.
-     * @param descCols Columns in descending order.
-     * @param idxs Indexes;
-     */
-    public DbTable(String schema, String tbl, Collection<DbColumn> cols, Set<String> ascCols, Set<String> descCols,
-        Map<String, Map<String, Boolean>> idxs) {
-        this.schema = schema;
-        this.tbl = tbl;
-        this.cols = cols;
-        this.ascCols = ascCols;
-        this.descCols = descCols;
-        this.idxs = idxs;
-    }
-
-    /**
-     * @return Schema name.
-     */
-    public String schema() {
-        return schema;
-    }
-
-    /**
-     * @return Table name.
-     */
-    public String table() {
-        return tbl;
-    }
-
-    /**
-     * @return Columns.
-     */
-    public Collection<DbColumn> columns() {
-        return cols;
-    }
-
-    /**
-     * @return Fields in ascending order
-     */
-    public Set<String> ascendingColumns() {
-        return ascCols;
-    }
-
-    /**
-     * @return Fields in descending order
-     */
-    public Set<String> descendingColumns() {
-        return descCols;
-    }
-
-    /**
-     * @return Indexes.
-     */
-    public Map<String, Map<String, Boolean>> indexes() {
-        return idxs;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1e3c77f6/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
deleted file mode 100644
index 17eb8b2..0000000
--- a/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/DB2MetadataDialect.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * 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.util.*;
-
-/**
- * DB2 specific metadata dialect.
- */
-public class DB2MetadataDialect extends JdbcMetadataDialect {
-    /** {@inheritDoc} */
-    @Override public Set<String> systemSchemas() {
-        return new HashSet<>(Arrays.asList("SYSIBM", "SYSCAT", "SYSSTAT", "SYSTOOLS"));
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1e3c77f6/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
deleted file mode 100644
index 0d17567..0000000
--- a/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/DatabaseMetadataDialect.java
+++ /dev/null
@@ -1,78 +0,0 @@
-/*
- * 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 org.apache.ignite.schema.parser.*;
-
-import java.sql.*;
-import java.util.*;
-
-/**
- * Base class for database metadata dialect.
- */
-public abstract class DatabaseMetadataDialect {
-    /**
-     * Gets tables from database.
-     *
-     * @param conn Database connection.
-     * @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;
-
-    /**
-     * @return Collection of database system schemas.
-     */
-    public Set<String> systemSchemas() {
-        return Collections.singleton("INFORMATION_SCHEMA");
-    }
-
-    /**
-     * Create table descriptor.
-     *
-     * @param schema Schema name.
-     * @param tbl Table name.
-     * @param cols Table columns.
-     * @param idxs Table indexes.
-     * @return New {@code DbTable} instance.
-     */
-    protected DbTable table(String schema, String tbl, Collection<DbColumn> cols, Map<String, Map<String, Boolean>>idxs) {
-        Set<String> ascCols = new HashSet<>();
-
-        Set<String> descCols = new HashSet<>();
-
-        for (Map<String, Boolean> idx : idxs.values()) {
-            if (idx.size() == 1)
-                for (Map.Entry<String, Boolean> idxCol : idx.entrySet()) {
-                    String colName = idxCol.getKey();
-
-                    Boolean desc = idxCol.getValue();
-
-                    if (desc != null) {
-                        if (desc)
-                            descCols.add(colName);
-                        else
-                            ascCols.add(colName);
-                    }
-                }
-        }
-
-        return new DbTable(schema, tbl, cols, ascCols, descCols, idxs);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1e3c77f6/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
deleted file mode 100644
index ab65e7a..0000000
--- a/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/JdbcMetadataDialect.java
+++ /dev/null
@@ -1,141 +0,0 @@
-/*
- * 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 org.apache.ignite.schema.parser.*;
-
-import java.sql.*;
-import java.util.*;
-
-/**
- * Metadata dialect that uses standard JDBC for reading metadata.
- */
-public class JdbcMetadataDialect extends DatabaseMetadataDialect {
-    /** */
-    private static final String[] TABLES_ONLY = {"TABLE"};
-
-    /** */
-    private static final String[] TABLES_AND_VIEWS = {"TABLE", "VIEW"};
-
-    /** Schema catalog index. */
-    private static final int TBL_CATALOG_IDX = 1;
-
-    /** Schema name index. */
-    private static final int TBL_SCHEMA_IDX = 2;
-
-    /** Table name index. */
-    private static final int TBL_NAME_IDX = 3;
-
-    /** Primary key column name index. */
-    private static final int PK_COL_NAME_IDX = 4;
-
-    /** Column name index. */
-    private static final int COL_NAME_IDX = 4;
-
-    /** Column data type index. */
-    private static final int COL_DATA_TYPE_IDX = 5;
-
-    /** Column nullable index. */
-    private static final int COL_NULLABLE_IDX = 11;
-
-    /** Index name index. */
-    private static final int IDX_NAME_IDX = 6;
-
-    /** Index column name index. */
-    private static final int IDX_COL_NAME_IDX = 9;
-
-    /** Index column descend index. */
-    private static final int IDX_ASC_OR_DESC_IDX = 10;
-
-    /** {@inheritDoc} */
-    @Override public Collection<DbTable> tables(Connection conn, 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);
-
-                // In case of MySql we should use catalog.
-                String schema = tblSchema != null ? tblSchema : tblCatalog;
-
-                // Skip system schemas.
-                if (sys.contains(schema))
-                    continue;
-
-                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<>();
-
-                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));
-                    }
-                }
-
-                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);
-
-                        String colName = idxRs.getString(IDX_COL_NAME_IDX);
-
-                        if (idxName == null || colName == null)
-                            continue;
-
-                        Map<String, Boolean> idx = idxs.get(idxName);
-
-                        if (idx == null) {
-                            idx = new LinkedHashMap<>();
-
-                            idxs.put(idxName, idx);
-                        }
-
-                        String askOrDesc = idxRs.getString(IDX_ASC_OR_DESC_IDX);
-
-                        Boolean desc = askOrDesc != null ? "D".equals(askOrDesc) : null;
-
-                        idx.put(colName, desc);
-                    }
-                }
-
-                tbls.add(table(schema, tblName, cols, idxs));
-            }
-        }
-
-        return tbls;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1e3c77f6/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
deleted file mode 100644
index 855c9f7..0000000
--- a/modules/schema-import/src/main/java/org/apache/ignite/schema/parser/dialect/OracleMetadataDialect.java
+++ /dev/null
@@ -1,281 +0,0 @@
-/*
- * 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 org.apache.ignite.schema.parser.*;
-
-import java.sql.*;
-import java.util.*;
-
-import static java.sql.Types.*;
-
-/**
- * Oracle specific metadata dialect.
- */
-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'" +
-        " ORDER BY a.owner, a.table_name, a.column_id";
-
-    /** SQL to get list of PRIMARY KEYS columns. */
-    private static final String SQL_PRIMARY_KEYS = "SELECT b.column_name" +
-        " FROM all_constraints a" +
-        "  INNER JOIN all_cons_columns b ON a.owner = b.owner AND a.constraint_name = b.constraint_name" +
-        " WHERE a.owner = ? and a.table_name = ? AND a.constraint_type = 'P'";
-
-    /** SQL to get indexes metadata. */
-    private static final String SQL_INDEXES = "select i.index_name, u.column_expression, i.column_name, i.descend" +
-        " FROM all_ind_columns i" +
-        " LEFT JOIN user_ind_expressions u on u.index_name = i.index_name and i.table_name = u.table_name" +
-        " WHERE i.index_owner = ? and i.table_name = ?" +
-        " ORDER BY i.index_name, i.column_position";
-
-    /** Owner index. */
-    private static final int OWNER_IDX = 1;
-
-    /** Table name index. */
-    private static final int TBL_NAME_IDX = 2;
-
-    /** Column name index. */
-    private static final int COL_NAME_IDX = 3;
-
-    /** Nullable index. */
-    private static final int NULLABLE_IDX = 4;
-
-    /** Data type index. */
-    private static final int DATA_TYPE_IDX = 5;
-
-    /** Numeric precision index. */
-    private static final int DATA_PRECISION_IDX = 6;
-
-    /** Numeric scale index. */
-    private static final int DATA_SCALE_IDX = 7;
-
-    /** Index name index. */
-    private static final int IDX_NAME_IDX = 1;
-
-    /** Index name index. */
-    private static final int IDX_EXPR_IDX = 2;
-
-    /** Index column name index. */
-    private static final int IDX_COL_NAME_IDX = 3;
-
-    /** Index column sort order index. */
-    private static final int IDX_COL_DESCEND_IDX = 4;
-
-    /**
-     * @param rs Result set with column type metadata from Oracle database.
-     * @return JDBC type.
-     * @throws SQLException If failed to decode type.
-     */
-    private int decodeType(ResultSet rs) throws SQLException {
-        switch (rs.getString(DATA_TYPE_IDX)) {
-            case "CHAR":
-            case "NCHAR":
-                return CHAR;
-
-            case "VARCHAR2":
-            case "NVARCHAR2":
-                return VARCHAR;
-
-            case "LONG":
-                return LONGVARCHAR;
-
-            case "LONG RAW":
-                return LONGVARBINARY;
-
-            case "FLOAT":
-                return FLOAT;
-
-            case "NUMBER":
-                int precision = rs.getInt(DATA_PRECISION_IDX);
-                int scale = rs.getInt(DATA_SCALE_IDX);
-
-                if (scale > 0) {
-                    if (scale < 4 && precision < 19)
-                        return FLOAT;
-
-                    if (scale > 4 || precision > 19)
-                        return DOUBLE;
-
-                    return NUMERIC;
-                }
-                else {
-                    if (precision < 1)
-                        return INTEGER;
-
-                    if (precision < 2)
-                        return BOOLEAN;
-
-                    if (precision < 4)
-                        return TINYINT;
-
-                    if (precision < 6)
-                        return SMALLINT;
-
-                    if (precision < 11)
-                        return INTEGER;
-
-                    if (precision < 20)
-                        return BIGINT;
-
-                    return NUMERIC;
-                }
-
-            case "DATE":
-                return DATE;
-
-            case "TIMESTAMP":
-                return TIMESTAMP;
-
-            case "BFILE":
-            case "BLOB":
-                return BLOB;
-
-            case "CLOB":
-            case "NCLOB":
-            case "XMLTYPE":
-                return CLOB;
-        }
-
-        return OTHER;
-    }
-
-    /**
-     * Retrieve primary key columns.
-     *
-     * @param stmt Prepared SQL statement to execute.
-     * @param owner DB owner.
-     * @param tbl Table name.
-     * @return Primary key columns.
-     * @throws SQLException If failed to retrieve primary key columns.
-     */
-    private Set<String> primaryKeys(PreparedStatement stmt, String owner, String tbl) throws SQLException {
-        Set<String> pkCols = new HashSet<>();
-
-        stmt.setString(1, owner);
-        stmt.setString(2, tbl);
-
-        try (ResultSet pkRs = stmt.executeQuery()) {
-            while(pkRs.next())
-                pkCols.add(pkRs.getString(1));
-        }
-
-        return pkCols;
-    }
-
-    /**
-     * Retrieve index columns.
-     *
-     * @param stmt Prepared SQL statement to execute.
-     * @param owner DB owner.
-     * @param tbl Table name.
-     * @return Index columns.
-     * @throws SQLException If failed to retrieve indexes columns.
-     */
-    private Map<String, Map<String, Boolean>> indexes(PreparedStatement stmt, String owner, String tbl)
-        throws SQLException {
-        Map<String, Map<String, Boolean>> idxs = new LinkedHashMap<>();
-
-        stmt.setString(1, owner);
-        stmt.setString(2, tbl);
-
-        try (ResultSet idxsRs = stmt.executeQuery()) {
-            while (idxsRs.next()) {
-                String idxName = idxsRs.getString(IDX_NAME_IDX);
-
-                Map<String, Boolean> idx = idxs.get(idxName);
-
-                if (idx == null) {
-                    idx = new LinkedHashMap<>();
-
-                    idxs.put(idxName, idx);
-                }
-
-                String expr = idxsRs.getString(IDX_EXPR_IDX);
-
-                String col = expr == null ? idxsRs.getString(IDX_COL_NAME_IDX) : expr.replaceAll("\"", "");
-
-                idx.put(col, "DESC".equals(idxsRs.getString(IDX_COL_DESCEND_IDX)));
-            }
-        }
-
-        return idxs;
-    }
-
-    /** {@inheritDoc} */
-    @Override public Collection<DbTable> tables(Connection conn, boolean tblsOnly) throws SQLException {
-        Collection<DbTable> tbls = new ArrayList<>();
-
-        PreparedStatement pkStmt = conn.prepareStatement(SQL_PRIMARY_KEYS);
-
-        PreparedStatement idxStmt = conn.prepareStatement(SQL_INDEXES);
-
-        try (Statement colsStmt = conn.createStatement()) {
-            Collection<DbColumn> cols = new ArrayList<>();
-
-            Set<String> pkCols = Collections.emptySet();
-            Map<String, Map<String, Boolean>> idxs = Collections.emptyMap();
-
-            String user = conn.getMetaData().getUserName().toUpperCase();
-
-            String sql = String.format(SQL_COLUMNS,
-                tblsOnly ? "INNER JOIN all_tables b on a.table_name = b.table_name" : "", user);
-
-            try (ResultSet colsRs = colsStmt.executeQuery(sql)) {
-                String prevSchema = "";
-                String prevTbl = "";
-
-                boolean first = true;
-
-                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 (changed) {
-                        if (first)
-                            first = false;
-                        else
-                            tbls.add(table(prevSchema, prevTbl, cols, idxs));
-
-                        prevSchema = owner;
-                        prevTbl = tbl;
-                        cols = new ArrayList<>();
-                        pkCols = primaryKeys(pkStmt, owner, tbl);
-                        idxs = indexes(idxStmt, owner, tbl);
-                    }
-
-                    String colName = colsRs.getString(COL_NAME_IDX);
-
-                    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));
-            }
-        }
-
-        return tbls;
-    }
-}