You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sqoop.apache.org by ja...@apache.org on 2015/11/05 18:40:58 UTC

[4/6] sqoop git commit: SQOOP-2595: Add Oracle connector to Sqoop 2

http://git-wip-us.apache.org/repos/asf/sqoop/blob/fa3c77b6/connector/connector-oracle-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/oracle/util/OracleConnectionFactory.java
----------------------------------------------------------------------
diff --git a/connector/connector-oracle-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/oracle/util/OracleConnectionFactory.java b/connector/connector-oracle-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/oracle/util/OracleConnectionFactory.java
new file mode 100644
index 0000000..3ebb0d4
--- /dev/null
+++ b/connector/connector-oracle-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/oracle/util/OracleConnectionFactory.java
@@ -0,0 +1,246 @@
+/**
+ * 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.sqoop.connector.jdbc.oracle.util;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.log4j.Logger;
+import org.apache.sqoop.connector.jdbc.oracle.OracleJdbcConnectorConstants;
+import org.apache.sqoop.connector.jdbc.oracle.configuration.ConnectionConfig;
+
+/**
+ * Create and initialize connections to Oracle RDBMS.
+ */
+public class OracleConnectionFactory {
+
+  protected OracleConnectionFactory() {
+  }
+
+  private static final Logger LOG =
+      Logger.getLogger(OracleConnectionFactory.class);
+
+  public static Connection makeConnection(ConnectionConfig config) throws SQLException {
+
+    String connectStr = config.connectionString;
+    String username = config.username;
+    String password = config.password;
+    Properties additionalProps = new Properties();
+    if(config.jdbcProperties != null) {
+      additionalProps.putAll(config.jdbcProperties);
+    }
+
+    Connection connection =
+        OracleConnectionFactory.createOracleJdbcConnection(
+            OracleJdbcConnectorConstants.ORACLE_JDBC_DRIVER_CLASS,
+            connectStr, username, password, additionalProps);
+    //TODO: This is from the other Oracle Manager
+//    if (username == null) {
+//      username = OracleManager.getSessionUser(connection);
+//    }
+    OracleUtilities.setCurrentSessionUser(username);
+    return connection;
+  }
+
+  public static Connection createOracleJdbcConnection(
+      String jdbcDriverClassName, String jdbcUrl, String username,
+      String password) throws SQLException {
+    Properties props = null;
+    return createOracleJdbcConnection(jdbcDriverClassName, jdbcUrl, username,
+        password, props);
+  }
+
+  public static Connection createOracleJdbcConnection(
+      String jdbcDriverClassName, String jdbcUrl, String username,
+      String password, Properties additionalProps) throws SQLException {
+
+    loadJdbcDriver(jdbcDriverClassName);
+    Connection connection =
+        createConnection(jdbcUrl, username, password, additionalProps);
+
+    // Only OraOopDBRecordReader will call initializeOracleConnection(), as
+    // we only need to initialize the session(s) prior to the mapper starting
+    // it's job.
+    // i.e. We don't need to initialize the sessions in order to get the
+    // table's data-files etc.
+
+    // initializeOracleConnection(connection, conf);
+
+    return connection;
+  }
+
+  private static void loadJdbcDriver(String jdbcDriverClassName) {
+
+    try {
+      Class.forName(jdbcDriverClassName);
+    } catch (ClassNotFoundException ex) {
+      String errorMsg =
+          "Unable to load the jdbc driver class : " + jdbcDriverClassName;
+      LOG.error(errorMsg);
+      throw new RuntimeException(errorMsg);
+    }
+  }
+
+  private static Connection createConnection(String jdbcUrl, String username,
+      String password, Properties additionalProps) throws SQLException {
+
+    Properties props = new Properties();
+    if (username != null) {
+      props.put("user", username);
+    }
+
+    if (password != null) {
+      props.put("password", password);
+    }
+
+    if (additionalProps != null && additionalProps.size() > 0) {
+      props.putAll(additionalProps);
+    }
+
+    OracleUtilities.checkJavaSecurityEgd();
+
+    try {
+      Connection result = DriverManager.getConnection(jdbcUrl, props);
+      result.setAutoCommit(false);
+      return result;
+    } catch (SQLException ex) {
+      String errorMsg = String.format(
+        "Unable to obtain a JDBC connection to the URL \"%s\" as user \"%s\": ",
+        jdbcUrl, (username != null) ? username : "[null]");
+      LOG.error(errorMsg, ex);
+      throw ex;
+    }
+  }
+
+  public static void initializeOracleConnection(Connection connection,
+      ConnectionConfig config) throws SQLException {
+
+    connection.setTransactionIsolation(Connection.TRANSACTION_READ_COMMITTED);
+
+    connection.setAutoCommit(false);
+
+    OracleQueries.setConnectionTimeZone(connection, config.timeZone);
+
+    setSessionClientInfo(connection, config);
+
+    OracleQueries.setJdbcFetchSize(connection, config.fetchSize);
+
+    executeOraOopSessionInitializationStatements(connection,
+        config.initializationStatements);
+  }
+
+  public static void setSessionClientInfo(Connection connection,
+      ConnectionConfig config) {
+
+    String sql = "";
+    try {
+      sql =
+          "begin \n"
+              + "  dbms_application_info.set_module(module_name => "
+              + "'%s', action_name => '%s'); \n"
+              + "end;";
+
+      String oracleSessionActionName = config.actionName;
+
+      sql =
+          String.format(sql,
+              OracleJdbcConnectorConstants.ORACLE_SESSION_MODULE_NAME,
+              oracleSessionActionName);
+
+      Statement statement = connection.createStatement();
+      statement.execute(sql);
+      LOG.info("Initializing Oracle session with SQL :\n" + sql);
+    } catch (Exception ex) {
+      LOG.error(String.format("An error occurred while attempting to execute "
+          + "the following Oracle session-initialization statement:" + "\n%s"
+          + "\nError:" + "\n%s", sql, ex.getMessage()));
+    }
+  }
+
+  public static void executeOraOopSessionInitializationStatements(
+      Connection connection, String sessionInitializationStatements) {
+    String statementsStr = sessionInitializationStatements;
+    if(StringUtils.isEmpty(statementsStr)) {
+      statementsStr =OracleJdbcConnectorConstants.
+          ORACLE_SESSION_INITIALIZATION_STATEMENTS_DEFAULT;
+    }
+
+    List<String> statements =
+        parseOraOopSessionInitializationStatements(statementsStr);
+
+    if (statements.size() == 0) {
+      LOG.warn("No Oracle 'session initialization' statements were found to "
+              + "execute.");
+    } else {
+      for (String statement : statements) {
+        try {
+          connection.createStatement().execute(statement);
+          LOG.info("Initializing Oracle session with SQL : " + statement);
+        } catch (Exception ex) {
+          LOG.error(String.format(
+              "An error occurred while attempting to execute "
+                  + "the following Oracle session-initialization statement:"
+                  + "\n%s" + "\nError:" + "\n%s", statement, ex.getMessage()));
+        }
+      }
+    }
+  }
+
+  public static List<String> parseOraOopSessionInitializationStatements(
+      String sessionInitializationStatements) {
+
+    ArrayList<String> result = new ArrayList<String>();
+
+    if (sessionInitializationStatements != null
+        && !sessionInitializationStatements.isEmpty()) {
+      String[] initializationStatements =
+          sessionInitializationStatements.split(";");
+      for (String initializationStatement : initializationStatements) {
+        initializationStatement = initializationStatement.trim();
+        if (initializationStatement != null
+            && !initializationStatement.isEmpty()
+            && !initializationStatement
+                .startsWith(OracleJdbcConnectorConstants.Oracle.
+                    ORACLE_SQL_STATEMENT_COMMENT_TOKEN)) {
+
+          LOG.debug(String
+              .format(
+                  "initializationStatement (quoted & pre-expression "
+                  + "evaluation) = \"%s\"",
+                  initializationStatement));
+
+          //TODO: Not supported in Sqoop 2?
+          /*initializationStatement =
+              OracleUtilities.replaceConfigurationExpression(
+                  initializationStatement, conf);*/
+
+          result.add(initializationStatement);
+        }
+      }
+    }
+    return result;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/fa3c77b6/connector/connector-oracle-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/oracle/util/OracleDataChunk.java
----------------------------------------------------------------------
diff --git a/connector/connector-oracle-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/oracle/util/OracleDataChunk.java b/connector/connector-oracle-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/oracle/util/OracleDataChunk.java
new file mode 100644
index 0000000..5b24fe3
--- /dev/null
+++ b/connector/connector-oracle-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/oracle/util/OracleDataChunk.java
@@ -0,0 +1,48 @@
+/**
+ * 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.sqoop.connector.jdbc.oracle.util;
+
+import org.apache.sqoop.job.etl.Partition;
+
+/**
+ * How data should be split between mappers.
+ */
+public abstract class OracleDataChunk extends Partition {
+
+  private String id;
+
+  public abstract long getNumberOfBlocks();
+
+  public String getWhereClause() {
+    return "1=1";
+  }
+
+  public String getPartitionClause() {
+    return "";
+  }
+
+  public String getId() {
+    return id;
+  }
+
+  public void setId(String newId) {
+    this.id = newId;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/fa3c77b6/connector/connector-oracle-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/oracle/util/OracleDataChunkExtent.java
----------------------------------------------------------------------
diff --git a/connector/connector-oracle-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/oracle/util/OracleDataChunkExtent.java b/connector/connector-oracle-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/oracle/util/OracleDataChunkExtent.java
new file mode 100644
index 0000000..2f794c2
--- /dev/null
+++ b/connector/connector-oracle-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/oracle/util/OracleDataChunkExtent.java
@@ -0,0 +1,109 @@
+/**
+ * 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.sqoop.connector.jdbc.oracle.util;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.sqoop.connector.jdbc.oracle.OracleJdbcConnectorConstants;
+
+/**
+ * Data should be split by extent for ROWID scans.
+ */
+public class OracleDataChunkExtent extends OracleDataChunk {
+
+  private int oracleDataObjectId;
+  private int relativeDatafileNumber;
+  private long startBlockNumber;
+  private long finishBlockNumber;
+
+  public OracleDataChunkExtent() {
+
+  }
+
+  public OracleDataChunkExtent(String id, int oracleDataObjectId,
+      int relativeDatafileNumber, long startBlockNumber,
+      long finishBlockNumber) {
+
+    this.setId(id);
+    this.oracleDataObjectId = oracleDataObjectId;
+    this.relativeDatafileNumber = relativeDatafileNumber;
+    this.startBlockNumber = startBlockNumber;
+    this.finishBlockNumber = finishBlockNumber;
+  }
+
+  @Override
+  public String getWhereClause() {
+    return String.format(
+        "(rowid >= dbms_rowid.rowid_create(%d, %d, %d, %d, %d)",
+        OracleJdbcConnectorConstants.Oracle.ROWID_EXTENDED_ROWID_TYPE,
+        this.oracleDataObjectId, this.relativeDatafileNumber,
+        this.startBlockNumber, 0)
+        + String.format(
+            " AND rowid <= dbms_rowid.rowid_create(%d, %d, %d, %d, %d))",
+            OracleJdbcConnectorConstants.Oracle.ROWID_EXTENDED_ROWID_TYPE,
+            this.oracleDataObjectId, this.relativeDatafileNumber,
+            this.finishBlockNumber,
+            OracleJdbcConnectorConstants.Oracle.ROWID_MAX_ROW_NUMBER_PER_BLOCK);
+  }
+
+  @Override
+  public void write(DataOutput output) throws IOException {
+    output.writeUTF(this.getId());
+    output.writeInt(this.oracleDataObjectId);
+    output.writeInt(this.relativeDatafileNumber);
+    output.writeLong(this.startBlockNumber);
+    output.writeLong(this.finishBlockNumber);
+  }
+
+  @Override
+  public void readFields(DataInput input) throws IOException {
+    this.setId(input.readUTF());
+    this.oracleDataObjectId = input.readInt();
+    this.relativeDatafileNumber = input.readInt();
+    this.startBlockNumber = input.readLong();
+    this.finishBlockNumber = input.readLong();
+  }
+
+  @Override
+  public long getNumberOfBlocks() {
+    if (this.finishBlockNumber == 0L && this.startBlockNumber == 0L) {
+      return 0;
+    } else {
+      return (this.finishBlockNumber - this.startBlockNumber) + 1L;
+    }
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append(String.format("\n\t%s = %s", "id", getId()));
+    sb.append(String.format("\n\t%s = %s",
+        "oracleDataObjectId", oracleDataObjectId));
+    sb.append(String.format("\n\t%s = %s",
+        "relativeDatafileNumber", relativeDatafileNumber));
+    sb.append(String.format("\n\t%s = %s",
+        "startBlockNumber", startBlockNumber));
+    sb.append(String.format("\n\t%s = %s",
+        "finishBlockNumber", finishBlockNumber));
+    return sb.toString();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/fa3c77b6/connector/connector-oracle-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/oracle/util/OracleDataChunkPartition.java
----------------------------------------------------------------------
diff --git a/connector/connector-oracle-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/oracle/util/OracleDataChunkPartition.java b/connector/connector-oracle-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/oracle/util/OracleDataChunkPartition.java
new file mode 100644
index 0000000..0a47e1f
--- /dev/null
+++ b/connector/connector-oracle-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/oracle/util/OracleDataChunkPartition.java
@@ -0,0 +1,85 @@
+/**
+ * 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.sqoop.connector.jdbc.oracle.util;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+/**
+ * Data should be split by partition.
+ */
+public class OracleDataChunkPartition extends OracleDataChunk {
+
+  private boolean isSubPartition;
+  private long blocks;
+
+  public OracleDataChunkPartition() {
+
+  }
+
+  OracleDataChunkPartition(String partitionName, boolean isSubPartition,
+      long blocks) {
+    this.setId(partitionName);
+    this.isSubPartition = isSubPartition;
+    this.blocks = blocks;
+  }
+
+  @Override
+  public long getNumberOfBlocks() {
+    return this.blocks;
+  }
+
+  @Override
+  public void write(DataOutput output) throws IOException {
+    output.writeUTF(this.getId());
+    output.writeBoolean(this.isSubPartition);
+    output.writeLong(this.blocks);
+  }
+
+  @Override
+  public void readFields(DataInput input) throws IOException {
+    this.setId(input.readUTF());
+    this.isSubPartition = input.readBoolean();
+    this.blocks = input.readLong();
+  }
+
+  @Override
+  public String getPartitionClause() {
+    StringBuilder sb = new StringBuilder();
+    sb.append(" ");
+    if (this.isSubPartition) {
+      sb.append("SUBPARTITION");
+    } else {
+      sb.append("PARTITION");
+    }
+    sb.append("(\"").append(this.getId()).append("\")");
+    return sb.toString();
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append(String.format("\n\t%s = %s", "id", getId()));
+    sb.append(String.format("\n\t%s = %s", "isSubPartition", isSubPartition));
+    sb.append(String.format("\n\t%s = %s", "blocks", blocks));
+    return sb.toString();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/fa3c77b6/connector/connector-oracle-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/oracle/util/OracleGenerics.java
----------------------------------------------------------------------
diff --git a/connector/connector-oracle-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/oracle/util/OracleGenerics.java b/connector/connector-oracle-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/oracle/util/OracleGenerics.java
new file mode 100644
index 0000000..0ecf587
--- /dev/null
+++ b/connector/connector-oracle-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/oracle/util/OracleGenerics.java
@@ -0,0 +1,64 @@
+/**
+ * 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.sqoop.connector.jdbc.oracle.util;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+/**
+ * Generic class to hold list of objects.
+ */
+public class OracleGenerics {
+
+  /**
+   * Generic class to hold list of objects.
+   */
+  public static class ObjectList<T> {
+
+    private List<T> objects;
+
+    public ObjectList() {
+
+      this.objects = new ArrayList<T>();
+    }
+
+    public void add(T item) {
+
+      this.objects.add(item);
+    }
+
+    public int size() {
+
+      return this.objects.size();
+    }
+
+    public T get(int index) {
+
+      return this.objects.get(index);
+    }
+
+    public Iterator<T> iterator() {
+
+      return this.objects.iterator();
+    }
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/fa3c77b6/connector/connector-oracle-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/oracle/util/OracleJdbcUrl.java
----------------------------------------------------------------------
diff --git a/connector/connector-oracle-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/oracle/util/OracleJdbcUrl.java b/connector/connector-oracle-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/oracle/util/OracleJdbcUrl.java
new file mode 100644
index 0000000..65d0092
--- /dev/null
+++ b/connector/connector-oracle-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/oracle/util/OracleJdbcUrl.java
@@ -0,0 +1,244 @@
+/**
+ * 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.sqoop.connector.jdbc.oracle.util;
+
+import org.apache.sqoop.connector.jdbc.oracle.util.OracleUtilities.JdbcOracleThinConnection;
+import org.apache.sqoop.connector.jdbc.oracle.util.OracleUtilities
+         .JdbcOracleThinConnectionParsingError;
+
+/**
+ * Parses the Oracle connection string.
+ */
+public class OracleJdbcUrl {
+
+  private String jdbcConnectString;
+
+  public OracleJdbcUrl(String jdbcConnectString) {
+
+    if (jdbcConnectString == null) {
+      throw new IllegalArgumentException(
+          "The jdbcConnectionString argument must not be null.");
+    }
+
+    if (jdbcConnectString.isEmpty()) {
+      throw new IllegalArgumentException(
+          "The jdbcConnectionString argument must not be empty.");
+    }
+
+    this.jdbcConnectString = jdbcConnectString;
+  }
+
+  public JdbcOracleThinConnection parseJdbcOracleThinConnectionString()
+      throws JdbcOracleThinConnectionParsingError {
+
+    /*
+     * http://wiki.oracle.com/page/JDBC
+     *
+     * There are different flavours of JDBC connections for Oracle, including:
+     * Thin E.g. jdbc:oracle:thin:@localhost.locadomain:1521:orcl
+     *
+     * A pure Java driver used on the client side that does not need an Oracle
+     * client installation. It is recommended that you use this driver unless
+     * you need support for non-TCP/IP networks because it provides for maximum
+     * portability and performance.
+     *
+     * Oracle Call Interface driver (OCI). E.g. jdbc:oracle:oci8:@orcl.world
+     * //<- "orcl.world" is a TNS entry
+     *
+     * This uses the Oracle client installation libraries and interfaces. If you
+     * want to support connection pooling or client side caching of requests,
+     * use this driver. You will also need this driver if you are using
+     * transparent application failover (TAF) from your application as well as
+     * strong authentication like Kerberos and PKI certificates.
+     *
+     * JDBC-ODBC bridge. E.g. jdbc:odbc:mydatabase //<- "mydatabase" is an ODBC
+     * data source.
+     *
+     * This uses the ODBC driver in Windows to connect to the database.
+     */
+
+    String hostName = null;
+    int port = 0;
+    String sid = null;
+    String service = null;
+
+    String jdbcUrl = this.jdbcConnectString.trim();
+
+    // If there are any parameters included at the end of the connection URL,
+    // let's remove them now...
+    int paramsIdx = jdbcUrl.indexOf("?");
+    if (paramsIdx > -1) {
+      jdbcUrl = jdbcUrl.substring(0, paramsIdx);
+    }
+
+    /*
+     * The format of an Oracle jdbc URL is one of:
+     * jdbc:oracle:<driver-type>:@tnsname - for tnsname based login
+     * jdbc:oracle:<driver-type>:@<host>:<port>:<sid>
+     * jdbc:oracle:<driver-type>:@<host>:<port>/<service>
+     * jdbc:oracle:<driver-type>:@<host>:<port>/<service>?<parameters>
+     * jdbc:oracle:<driver-type>:@//<host>:<port>/<service>
+     * jdbc:oracle:<driver-type>:@//<host>:<port>/<service>?<parameters>
+     */
+
+    // Split the URL on its ":" characters...
+    String[] jdbcFragments = jdbcUrl.trim().split(":");
+
+    // Clean up each fragment of the URL...
+    for (int idx = 0; idx < jdbcFragments.length; idx++) {
+      jdbcFragments[idx] = jdbcFragments[idx].trim();
+    }
+
+    // Check we can proceed...
+    if (jdbcFragments.length < 4 || jdbcFragments.length > 6) {
+      throw new JdbcOracleThinConnectionParsingError(
+        String.format(
+          "There should be 4, 5 or 6 colon-separated pieces of data in the "
+        + "JDBC URL, such as:\n\tjdbc:oracle:<driver-type>:@tnsname\n"
+        + "\tjdbc:oracle:<driver-type>:@<host>:<port>:<sid>\n"
+        + "\tjdbc:oracle:<driver-type>:@<host>:<port>/<service>\n"
+        + "\tjdbc:oracle:<driver-type>:@<host>:<port>/<service>?<parameters>\n"
+        + "The JDBC URL specified was:\n"
+        + "%s\n"
+        + "which contains %d pieces of colon-separated data.",
+                  this.jdbcConnectString, jdbcFragments.length));
+    }
+
+    // jdbc
+    if (!jdbcFragments[0].equalsIgnoreCase("jdbc")) {
+      throw new JdbcOracleThinConnectionParsingError(
+          "The first item in the colon-separated JDBC URL must be \"jdbc\".");
+    }
+
+    // jdbc:oracle
+    if (!jdbcFragments[1].equalsIgnoreCase("oracle")) {
+      throw new JdbcOracleThinConnectionParsingError(
+        "The second item in the colon-separated JDBC URL must be \"oracle\".");
+    }
+
+    // jdbc:oracle:thin
+    if (!jdbcFragments[2].equalsIgnoreCase("thin")) {
+      throw new JdbcOracleThinConnectionParsingError(
+          String
+              .format(
+                  "The Oracle \"thin\" JDBC driver is not being used.\n"
+                      + "The third item in the colon-separated JDBC URL must "
+                      + "be \"thin\", not \"%s\".",
+                  jdbcFragments[2]));
+    }
+
+    // jdbc:oracle:thin:@<host>
+    hostName = jdbcFragments[3];
+    if (hostName.isEmpty() || hostName.equalsIgnoreCase("@")) {
+      throw new JdbcOracleThinConnectionParsingError(
+          "The fourth item in the colon-separated JDBC URL (the host name) "
+          + "must not be empty.");
+    }
+
+    if (!hostName.startsWith("@")) {
+      throw new JdbcOracleThinConnectionParsingError(
+          "The fourth item in the colon-separated JDBC URL (the host name) "
+          + "must a prefixed with the \"@\" character.");
+    }
+
+    String portStr = "";
+    String tnsName = "";
+
+    switch (jdbcFragments.length) {
+      case 6:
+        // jdbc:oracle:<driver-type>:@<host>:<port>:<sid>
+        portStr = jdbcFragments[4];
+        sid = jdbcFragments[5];
+        break;
+
+      case 5:
+        // jdbc:oracle:<driver-type>:@<host>:<port>/<service>
+        String[] portAndService = jdbcFragments[4].split("/");
+        if (portAndService.length != 2) {
+          throw new JdbcOracleThinConnectionParsingError(
+              "The fifth colon-separated item in the JDBC URL "
+              + "(<port>/<service>) must contain two items "
+              + "separated by a \"/\".");
+        }
+        portStr = portAndService[0].trim();
+        service = portAndService[1].trim();
+        break;
+
+      case 4:
+        // jdbc:oracle:<driver-type>:@tnsname
+        tnsName = jdbcFragments[3].trim();
+        break;
+
+      default:
+        throw new JdbcOracleThinConnectionParsingError("Internal error parsing "
+            + "JDBC connection string.");
+    }
+
+    if (jdbcFragments.length > 4) {
+      if (portStr.isEmpty()) {
+        throw new JdbcOracleThinConnectionParsingError(
+            "The fifth item in the colon-separated JDBC URL (the port) must not"
+            + " be empty.");
+      }
+
+      try {
+        port = Integer.parseInt(portStr);
+      } catch (NumberFormatException ex) {
+        throw new JdbcOracleThinConnectionParsingError(
+            String
+                .format(
+                    "The fifth item in the colon-separated JDBC URL (the port) "
+                    + "must be a valid number.\n"
+                    + "\"%s\" could not be parsed as an integer.", portStr));
+      }
+
+      if (port <= 0) {
+        throw new JdbcOracleThinConnectionParsingError(
+            String
+                .format(
+                    "The fifth item in the colon-separated JDBC URL (the port) "
+                    + "must be greater than zero.\n"
+                        + "\"%s\" was specified.", portStr));
+      }
+    }
+
+    if (sid == null && service == null && tnsName == null) {
+      throw new JdbcOracleThinConnectionParsingError(
+        "The JDBC URL does not contain a SID or SERVICE. The URL should look "
+      + "like one of these:\n\tjdbc:oracle:<driver-type>:@tnsname\n"
+      + "\tjdbc:oracle:<driver-type>:@<host>:<port>:<sid>\n"
+      + "\tjdbc:oracle:<driver-type>:@<host>:<port>/<service>\n"
+      + "\tjdbc:oracle:<driver-type>:@<host>:<port>/<service>?<parameters>\n"
+      + "\tjdbc:oracle:<driver-type>:@//<host>:<port>/<service>\n"
+      + "\tjdbc:oracle:<driver-type>:@<host>:<port>/<service>?<parameters>");
+    }
+
+    // Remove the "@" prefix of the hostname
+    JdbcOracleThinConnection result =
+        new JdbcOracleThinConnection(hostName.replaceFirst("^[@][/]{0,2}", "")
+            , port, sid, service, tnsName.replaceFirst("^[@][/]{0,2}", ""));
+
+    return result;
+  }
+
+  public String getConnectionUrl() {
+    return this.jdbcConnectString;
+  }
+
+}