This is an automated email from the ASF dual-hosted git repository.
machristie pushed a commit to branch airavata-2938-change-db-initialization-in-registry-server-to-use-registry-refactoring-code
in repository https://gitbox.apache.org/repos/asf/airavata.git
The following commit(s) were added to refs/heads/airavata-2938-change-db-initialization-in-registry-server-to-use-registry-refactoring-code by this push:
new 381ee48 AIRAVATA-2938 Convert sharing registry to use common DBInitializer
381ee48 is described below
commit 381ee48fe8ead1701683ca3d88a9d73c75db00af
Author: Marcus Christie <machristie@apache.org>
AuthorDate: Thu Jan 24 17:05:21 2019 -0500
AIRAVATA-2938 Convert sharing registry to use common DBInitializer
---
.../airavata/common/utils/DBInitializer.java | 4 +-
.../sharing/registry/db/utils/ConnectionPool.java | 381 ---------------------
.../sharing/registry/db/utils/DatabaseCreator.java | 352 -------------------
.../sharing/registry/db/utils/JPAUtils.java | 120 +------
.../sharing/registry/db/utils/JdbcStorage.java | 174 ----------
.../db/utils/SharingRegistryDBInitConfig.java | 46 +++
.../db/utils/SharingRegistryJDBCConfig.java | 61 ++++
.../registry/server/SharingRegistryServer.java | 23 +-
.../server/SharingRegistryServerHandler.java | 11 +-
.../registry/SharingRegistryServerHandlerTest.java | 14 +-
.../registry/SharingRegistryServiceTest.java | 11 +-
.../airavata/sharing/registry/util/Initialize.java | 296 ----------------
12 files changed, 149 insertions(+), 1344 deletions(-)
diff --git a/modules/commons/src/main/java/org/apache/airavata/common/utils/DBInitializer.java b/modules/commons/src/main/java/org/apache/airavata/common/utils/DBInitializer.java
index 7fddf07..ed9ddef 100644
--- a/modules/commons/src/main/java/org/apache/airavata/common/utils/DBInitializer.java
+++ b/modules/commons/src/main/java/org/apache/airavata/common/utils/DBInitializer.java
@@ -55,10 +55,10 @@ public class DBInitializer {
Connection conn = null;
try {
DBUtil dbUtil = new DBUtil(jdbcConfig);
- conn = dbUtil.getConnection();
- if (DatabaseCreator.getDatabaseType(conn) == DatabaseCreator.DatabaseType.derby && isDerbyStartEnabled()) {
+ if (DatabaseCreator.getDatabaseType(jdbcConfig.getURL()) == DatabaseCreator.DatabaseType.derby && isDerbyStartEnabled()) {
startDerbyInServerMode();
}
+ conn = dbUtil.getConnection();
if (!DatabaseCreator.isDatabaseStructureCreated(checkTableName, conn)) {
DatabaseCreator.createRegistryDatabase(initScriptPrefix, conn);
logger.info("New Database created from " + initScriptPrefix + " !!!");
diff --git a/modules/sharing-registry/sharing-registry-server/src/main/java/org/apache/airavata/sharing/registry/db/utils/ConnectionPool.java b/modules/sharing-registry/sharing-registry-server/src/main/java/org/apache/airavata/sharing/registry/db/utils/ConnectionPool.java
deleted file mode 100644
index aa59eb0..0000000
--- a/modules/sharing-registry/sharing-registry-server/src/main/java/org/apache/airavata/sharing/registry/db/utils/ConnectionPool.java
+++ /dev/null
@@ -1,381 +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.airavata.sharing.registry.db.utils;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import javax.sql.DataSource;
-import java.sql.Connection;
-import java.sql.DriverManager;
-import java.sql.SQLException;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Stack;
-import java.util.concurrent.Semaphore;
-
-
-/**
- * A class for preallocating, recycling, and managing JDBC connections.
- */
-public class ConnectionPool {
- private static final Logger logger = LoggerFactory.getLogger(ConnectionPool.class);
-
- private long MAX_IDLE_TIME = 5 * 60 * 1000; // 5 minutes
-
- private String driver;
- private String url;
- private String username;
- private String password;
- private String jdbcUrl;
-
- private int maxConnections;
-
- private boolean autoCommit = true;
- private boolean waitIfBusy;
-
- private Semaphore needConnection = new Semaphore(0);
- private boolean stop;
-
- private Stack<Connection> availableConnections;
- private Stack<Connection> busyConnections;
-
- private HashMap<Connection, Long> lastAccessTimeRecord = new HashMap<Connection, Long>();
-
- private String urlType = "";
-
- private DataSource datasource;
-
- private int transactionIsolation = Connection.TRANSACTION_NONE;
-
- private Thread clenupThread;
- private Thread producerThread;
-
- public ConnectionPool(String driver, String url, String username, String password, int initialConnections,
- int maxConnections, boolean waitIfBusy) throws SQLException {
- this.driver = driver;
- this.url = url;
- this.username = username;
- this.password = password;
- this.urlType = "speratedURL";
- initialize(initialConnections, maxConnections, waitIfBusy);
- }
-
- public ConnectionPool(String driver, String jdbcUrl, int initialConnections, int maxConnections,
- boolean waitIfBusy, boolean autoCommit, int transactionIsolation) throws SQLException {
- this.driver = driver;
- this.jdbcUrl = jdbcUrl;
- this.urlType = "simpleURL";
- this.autoCommit = autoCommit;
- this.transactionIsolation = transactionIsolation;
- initialize(initialConnections, maxConnections, waitIfBusy);
- }
-
- public ConnectionPool(String driver, String jdbcUrl, int initialConnections, int maxConnections, boolean waitIfBusy)
- throws SQLException {
- this.driver = driver;
- this.jdbcUrl = jdbcUrl;
- this.urlType = "simpleURL";
- initialize(initialConnections, maxConnections, waitIfBusy);
- }
-
- public ConnectionPool(DataSource dataSource, int initialConnections, int maxConnections, boolean waitIfBusy)
- throws SQLException {
- this.urlType = "dataSource";
- this.datasource = dataSource;
- initialize(initialConnections, maxConnections, waitIfBusy);
- }
-
- /**
- * Check if this connection pool is auto commit or not
- *
- * @return
- */
- public boolean isAutoCommit() {
- return this.autoCommit;
- }
-
- private void initialize(int initialConnections, int maxConnections, boolean waitIfBusy) throws SQLException {
- this.maxConnections = maxConnections;
- this.waitIfBusy = waitIfBusy;
-
- int sizeOfConnections = (initialConnections > maxConnections) ? maxConnections : initialConnections;
-
- availableConnections = new Stack<Connection>();
- busyConnections = new Stack<Connection>();
-
- for (int i = 0; i < sizeOfConnections; i++) {
- Connection con = makeNewConnection();
- setTimeStamp(con);
- availableConnections.push(con);
-
- }
-
- producerThread = new Thread(new FillUpThread());
- producerThread.start();
-
- clenupThread = new Thread(new CleanUpThread());
- clenupThread.start();
- }
-
- public synchronized Connection getConnection() throws SQLException {
- if (!availableConnections.isEmpty()) {
- Connection existingConnection = availableConnections.pop();
-
- // If connection on available list is closed (e.g.,
- // it timed out), then remove it from available list
- // and race for a connection again.
- if (existingConnection.isClosed()) {
- lastAccessTimeRecord.remove(existingConnection);
- // notifyAll for fairness
- notifyAll();
- } else {
- busyConnections.push(existingConnection);
- setTimeStamp(existingConnection);
- return existingConnection;
- }
- } else if (!waitIfBusy && busyConnections.size() >= maxConnections) {
- // You reached maxConnections limit and waitIfBusy flag is false.
- // Throw SQLException in such a case.
- throw new SQLException("Connection limit reached");
- } else {
-
- if (busyConnections.size() < maxConnections) {
- // available connection is empty, but total number of connection
- // doesn't reach maxConnection. Request for more connection
- needConnection.release();
- }
-
- try {
- // wait for free connection
- wait();
- } catch (InterruptedException ie) {
- }
- }
- // always race for connection forever
- return getConnection();
- }
-
- // This explicitly makes a new connection. Called in
- // the foreground when initializing the ConnectionPool,
- // and called in the background when running.
- private Connection makeNewConnection() throws SQLException {
- try {
- // Load database driver if not already loaded
- Class.forName(driver);
- Connection connection;
- // Establish network connection to database
- if (urlType.equals("speratedURL")) {
- connection = DriverManager.getConnection(url, username, password);
- } else if (urlType.equals("simpleURL")) {
- connection = DriverManager.getConnection(jdbcUrl);
- } else { // if(urlType.equals("dataSource")){
- connection = datasource.getConnection();
- }
- connection.setTransactionIsolation(this.transactionIsolation);
- connection.setAutoCommit(this.autoCommit);
- return connection;
- } catch (ClassNotFoundException cnfe) {
- // Simplify try/catch blocks of people using this by
- // throwing only one exception type.
- throw new SQLException("Can't find class for driver: " + driver);
- }
- }
-
- private synchronized void fillUpConnection(Connection conn) {
- setTimeStamp(conn);
- availableConnections.push(conn);
-
- // notify all since new connection is created
- notifyAll();
- }
-
- private void setTimeStamp(Connection connection) {
- lastAccessTimeRecord.put(connection, System.currentTimeMillis());
- }
-
- // The database connection cannot be left idle for too long, otherwise TCP
- // connection will be broken.
- /**
- * From http://forums.mysql.com/read.php?39,28450,57460#msg-57460 Okay, then it looks like wait_timeout on the
- * server is killing your connection (it is set to 8 hours of idle time by default). Either set that value higher on
- * your server, or configure your connection pool to not hold connections idle that long (I prefer the latter). Most
- * folks I know that run MySQL with a connection pool in high-load production environments only let connections sit
- * idle for a matter of minutes, since it only takes a few milliseconds to open a connection, and the longer one
- * sits idle the more chance it will go "bad" because of a network hiccup or the MySQL server being restarted.
- *
- * @throws java.sql.SQLException
- */
- private boolean isConnectionStale(Connection connection) {
- long currentTime = System.currentTimeMillis();
- long lastAccess = lastAccessTimeRecord.get(connection);
- if (currentTime - lastAccess > MAX_IDLE_TIME) {
- return true;
- } else
- return false;
- }
-
- private synchronized void closeStaleConnections() {
- // close idle connections
- Iterator<Connection> iter = availableConnections.iterator();
- while (iter.hasNext()) {
- Connection existingConnection = iter.next();
- if (isConnectionStale(existingConnection)) {
- try {
- existingConnection.close();
- iter.remove();
- } catch (SQLException sql) {
- logger.error(sql.getMessage(), sql);
- }
- }
- }
- // close busy connections that have been checked out for too long.
- // This should not happen since this means program has bug for not
- // releasing connections .
- iter = busyConnections.iterator();
- while (iter.hasNext()) {
- Connection busyConnection = iter.next();
- if (isConnectionStale(busyConnection)) {
- try {
- busyConnection.close();
- iter.remove();
- logger.warn("****Connection has checked out too long. Forced release. Check the program for calling release connection [free(Connection) method]");
- } catch (SQLException sql) {
- logger.error(sql.getMessage(), sql);
- }
- }
- }
- }
-
- public synchronized void free(Connection connection) {
- busyConnections.removeElement(connection);
- availableConnections.addElement(connection);
- // Wake up threads that are waiting for a connection
- notifyAll();
- }
-
- /**
- * Close all the connections. Use with caution: be sure no connections are in use before calling. Note that you are
- * not <I>required</I> to call this when done with a ConnectionPool, since connections are guaranteed to be closed
- * when garbage collected. But this method gives more control regarding when the connections are closed.
- */
- public synchronized void dispose() {
- logger.info("Connection Pool Shutting down");
-
- // stop clean up thread
- this.stop = true;
- this.clenupThread.interrupt();
-
- // stop producer up thread
- this.producerThread.interrupt();
-
- // close all connection
- closeConnections(availableConnections);
- availableConnections = new Stack<Connection>();
- closeConnections(busyConnections);
- busyConnections = new Stack<Connection>();
- lastAccessTimeRecord.clear();
-
- logger.info("All connection is closed");
-
- try {
- this.clenupThread.join();
- this.producerThread.join();
- } catch (Exception e) {
- logger.error("Cannot shutdown cleanup thread", e);
- }
-
- logger.info("Connection Pool Shutdown");
- }
-
- private void closeConnections(Stack<Connection> connections) {
- while (!connections.isEmpty()) {
- Connection connection = connections.pop();
- try {
- if (!connection.isClosed()) {
- connection.close();
- }
- } catch (SQLException sqle) {
- // Ignore errors; garbage collect anyhow
- logger.warn(sqle.getMessage());
- }
- }
- }
-
- public synchronized String toString() {
- String info = "ConnectionPool(" + url + "," + username + ")" + ", available=" + availableConnections.size()
- + ", busy=" + busyConnections.size() + ", max=" + maxConnections;
- return (info);
- }
-
- class CleanUpThread implements Runnable {
- public void run() {
- while (!stop) {
- try {
- Thread.sleep(MAX_IDLE_TIME);
- closeStaleConnections();
- } catch (InterruptedException e) {
- logger.info("Clean up thread is interrupted to close");
- }
- }
- }
- }
-
- class FillUpThread implements Runnable {
- public void run() {
- while (!stop) {
- try {
- // block until get
- needConnection.acquire();
-
- Connection conn = makeNewConnection();
- fillUpConnection(conn);
- } catch (SQLException e) {
- // cannot create connection (increase semaphore value back)
- needConnection.release();
- logger.error(e.getMessage(), e);
- } catch (InterruptedException e) {
- logger.info("Fill up thread is interrupted to close");
- break;
- }
- }
- }
- }
-
- public void shutdown() throws SQLException{
- for (Connection c : availableConnections) {
- try {
- c.close();
- } catch (SQLException e) {
- logger.error("Error while closing the connection", e);
- throw new SQLException("Error while closing the connection", e);
- }
- }
-
- for (Connection c : busyConnections) {
- try {
- c.close();
- } catch (SQLException e) {
- logger.error("Error while closing the connection", e);
- throw new SQLException("Error while closing the connection", e);
- }
- }
- }
-}
\ No newline at end of file
diff --git a/modules/sharing-registry/sharing-registry-server/src/main/java/org/apache/airavata/sharing/registry/db/utils/DatabaseCreator.java b/modules/sharing-registry/sharing-registry-server/src/main/java/org/apache/airavata/sharing/registry/db/utils/DatabaseCreator.java
deleted file mode 100644
index 9cc669b..0000000
--- a/modules/sharing-registry/sharing-registry-server/src/main/java/org/apache/airavata/sharing/registry/db/utils/DatabaseCreator.java
+++ /dev/null
@@ -1,352 +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.airavata.sharing.registry.db.utils;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.BufferedReader;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InputStreamReader;
-import java.sql.*;
-import java.util.StringTokenizer;
-
-/**
- * This class creates the database tables required for airavata with default configuration this
- * class creates derby database in server mode. User can specify required database in appropriate
- * properties files.
- */
-public class DatabaseCreator {
- private final static Logger logger = LoggerFactory.getLogger(DatabaseCreator.class);
-
- public enum DatabaseType {
- derby("(?i).*derby.*"), mysql("(?i).*mysql.*"), other("");
-
- private String pattern;
-
- private DatabaseType(String matchingPattern) {
- this.pattern = matchingPattern;
- }
-
- public String getMatchingPattern() {
- return this.pattern;
- }
- }
-
- private static DatabaseType[] supportedDatabase = new DatabaseType[] { DatabaseType.derby, DatabaseType.mysql };
-
- private static Logger log = LoggerFactory.getLogger(DatabaseCreator.class);
- private static final String delimiter = ";";
-
- /**
- * Creates database
- *
- * @throws Exception
- */
- public static void createRegistryDatabase(String prefix, Connection conn) throws Exception {
- createDatabase(prefix, conn);
- }
-
-
-
- /**
- * Checks whether database tables are created by using select * on given table name
- *
- * @param tableName
- * Table which should be existed
- * @return <code>true</core> if checkSQL is success, else <code>false</code> .
- */
- public static boolean isDatabaseStructureCreated(String tableName, Connection conn) {
- try {
-
- log.debug("Running a query to test the database tables existence.");
-
- // check whether the tables are already created with a query
- Statement statement = null;
- try {
- statement = conn.createStatement();
- ResultSet rs = statement.executeQuery("select * from " + tableName);
- if (rs != null) {
- rs.close();
- }
- } finally {
- try {
- if (statement != null) {
- statement.close();
- }
- } catch (SQLException e) {
- return false;
- }
- }
- } catch (SQLException e) {
- return false;
- }
-
- return true;
- }
-
- /**
- * executes given sql
- *
- * @param sql
- * @throws Exception
- */
- private static void executeSQL(String sql, Connection conn) throws Exception {
- // Check and ignore empty statements
- if ("".equals(sql.trim())) {
- return;
- }
-
- Statement statement = null;
- try {
- log.debug("SQL : " + sql);
-
- boolean ret;
- int updateCount = 0, updateCountTotal = 0;
- statement = conn.createStatement();
- ret = statement.execute(sql);
- updateCount = statement.getUpdateCount();
- do {
- if (!ret) {
- if (updateCount != -1) {
- updateCountTotal += updateCount;
- }
- }
- ret = statement.getMoreResults();
- if (ret) {
- updateCount = statement.getUpdateCount();
- }
- } while (ret);
-
- log.debug(sql + " : " + updateCountTotal + " rows affected");
-
- SQLWarning warning = conn.getWarnings();
- while (warning != null) {
- log.info(warning + " sql warning");
- warning = warning.getNextWarning();
- }
- conn.clearWarnings();
- } catch (SQLException e) {
- if (e.getSQLState().equals("X0Y32")) {
- // eliminating the table already exception for the derby
- // database
- log.info("Table Already Exists", e);
- } else {
- throw new Exception("Error occurred while executing : " + sql, e);
- }
- } finally {
- if (statement != null) {
- try {
- statement.close();
- } catch (SQLException e) {
- log.error("Error occurred while closing result set.", e);
- }
- }
- }
- }
-
- /**
- * computes relatational database type using database name
- *
- * @return DatabaseType
- * @throws Exception
- *
- */
- public static DatabaseType getDatabaseType(Connection conn) throws Exception {
- try {
- if (conn != null && (!conn.isClosed())) {
- DatabaseMetaData metaData = conn.getMetaData();
- String databaseProductName = metaData.getDatabaseProductName();
- return checkType(databaseProductName);
- }
- } catch (SQLException e) {
- String msg = "Failed to create Airavata database." + e.getMessage();
- log.error(msg, e);
- throw new Exception(msg, e);
- }
- return DatabaseType.other;
- }
-
- /**
- * Overloaded method with String input
- *
- * @return DatabaseType
- * @throws Exception
- *
- */
- public static DatabaseType getDatabaseType(String dbUrl) throws Exception {
- return checkType(dbUrl);
- }
-
- private static DatabaseType checkType(String text) throws Exception {
- try {
- if (text != null) {
- for (DatabaseType type : supportedDatabase) {
- if (text.matches(type.getMatchingPattern()))
- return type;
- }
- }
- String msg = "Unsupported database: " + text
- + ". Database will not be created automatically by the Airavata. "
- + "Please create the database using appropriate database scripts for " + "the database.";
- throw new Exception(msg);
-
- } catch (SQLException e) {
- String msg = "Failed to create Airavatadatabase." + e.getMessage();
- log.error(msg, e);
- throw new Exception(msg, e);
- }
- }
-
- /**
- * Get scripts location which is prefix + "-" + databaseType + ".sql"
- *
- * @param prefix
- * @param databaseType
- * @return script location
- */
- private static String getScriptLocation(String prefix, DatabaseType databaseType) {
- String scriptName = prefix + "-" + databaseType + ".sql";
- log.debug("Loading database script from :" + scriptName);
- return scriptName;
- }
-
- private static void createDatabase(String prefix, Connection conn) throws Exception {
- Statement statement = null;
- try {
- conn.setAutoCommit(false);
- statement = conn.createStatement();
- executeSQLScript(getScriptLocation(prefix, DatabaseCreator.getDatabaseType(conn)), conn);
- conn.commit();
- log.debug("Tables are created successfully.");
- } catch (SQLException e) {
- String msg = "Failed to create database tables for Airavata resource store. " + e.getMessage();
- log.error(msg, e);
- conn.rollback();
- throw new Exception(msg, e);
- } finally {
- conn.setAutoCommit(true);
- try {
- if (statement != null) {
- statement.close();
- }
- } catch (SQLException e) {
- log.error("Failed to close statement.", e);
- }
- }
- }
-
- private static void executeSQLScript(String dbscriptName, Connection conn) throws Exception {
- StringBuffer sql = new StringBuffer();
- BufferedReader reader = null;
-
- try {
- InputStream is = DatabaseCreator.class.getClassLoader().getResourceAsStream(dbscriptName);
- if(is == null) {
- logger.info("Script file not found at " + dbscriptName + ". Uses default database script file");
- DatabaseType databaseType = DatabaseCreator.getDatabaseType(conn);
- if(databaseType.equals(DatabaseType.derby)){
- is = DatabaseCreator.class.getClassLoader().getResourceAsStream("sharing-registry-derby.sql");
- }else if(databaseType.equals(DatabaseType.mysql)){
- is = DatabaseCreator.class.getClassLoader().getResourceAsStream("sharing-registry-mysql.sql");
- }
- }
- reader = new BufferedReader(new InputStreamReader(is));
- String line;
- while ((line = reader.readLine()) != null) {
- line = line.trim();
- if (line.startsWith("//")) {
- continue;
- }
- if (line.startsWith("--")) {
- continue;
- }
- StringTokenizer st = new StringTokenizer(line);
- if (st.hasMoreTokens()) {
- String token = st.nextToken();
- if ("REM".equalsIgnoreCase(token)) {
- continue;
- }
- }
- sql.append(" ").append(line);
-
- // SQL defines "--" as a comment to EOL
- // and in Oracle it may contain a hint
- // so we cannot just remove it, instead we must end it
- if (line.indexOf("--") >= 0) {
- sql.append("\n");
- }
- if ((checkStringBufferEndsWith(sql, delimiter))) {
- executeSQL(sql.substring(0, sql.length() - delimiter.length()), conn);
- sql.replace(0, sql.length(), "");
- }
- }
- // Catch any statements not followed by ;
- if (sql.length() > 0) {
- executeSQL(sql.toString(), conn);
- }
- } catch (IOException e) {
- log.error("Error occurred while executing SQL script for creating Airavata database", e);
- throw new Exception("Error occurred while executing SQL script for creating Airavata database", e);
-
- } finally {
- if (reader != null) {
- reader.close();
- }
- }
- }
-
- /**
- * Checks that a string buffer ends up with a given string. It may sound trivial with the existing JDK API but the
- * various implementation among JDKs can make those methods extremely resource intensive and perform poorly due to
- * massive memory allocation and copying. See
- *
- * @param buffer
- * the buffer to perform the check on
- * @param suffix
- * the suffix
- * @return <code>true</code> if the character sequence represented by the argument is a suffix of the character
- * sequence represented by the StringBuffer object; <code>false</code> otherwise. Note that the result will
- * be <code>true</code> if the argument is the empty string.
- */
- public static boolean checkStringBufferEndsWith(StringBuffer buffer, String suffix) {
- if (suffix.length() > buffer.length()) {
- return false;
- }
- // this loop is done on purpose to avoid memory allocation performance
- // problems on various JDKs
- // StringBuffer.lastIndexOf() was introduced in jdk 1.4 and
- // implementation is ok though does allocation/copying
- // StringBuffer.toString().endsWith() does massive memory
- // allocation/copying on JDK 1.5
- // See http://issues.apache.org/bugzilla/show_bug.cgi?id=37169
- int endIndex = suffix.length() - 1;
- int bufferIndex = buffer.length() - 1;
- while (endIndex >= 0) {
- if (buffer.charAt(bufferIndex) != suffix.charAt(endIndex)) {
- return false;
- }
- bufferIndex--;
- endIndex--;
- }
- return true;
- }
-}
\ No newline at end of file
diff --git a/modules/sharing-registry/sharing-registry-server/src/main/java/org/apache/airavata/sharing/registry/db/utils/JPAUtils.java b/modules/sharing-registry/sharing-registry-server/src/main/java/org/apache/airavata/sharing/registry/db/utils/JPAUtils.java
index 506a0d8..7052eac 100644
--- a/modules/sharing-registry/sharing-registry-server/src/main/java/org/apache/airavata/sharing/registry/db/utils/JPAUtils.java
+++ b/modules/sharing-registry/sharing-registry-server/src/main/java/org/apache/airavata/sharing/registry/db/utils/JPAUtils.java
@@ -22,16 +22,10 @@ package org.apache.airavata.sharing.registry.db.utils;
import org.apache.airavata.common.exception.ApplicationSettingsException;
import org.apache.airavata.common.utils.ServerSettings;
import org.apache.airavata.sharing.registry.models.SharingRegistryException;
-import org.apache.derby.drda.NetworkServerControl;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import javax.persistence.*;
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.URI;
-import java.sql.Connection;
-import java.sql.SQLException;
import java.util.HashMap;
import java.util.Map;
@@ -47,16 +41,6 @@ public class JPAUtils {
public static final String JPA_CACHE_SIZE = "jpa.cache.size";
public static final String JPA_CACHE_ENABLED = "cache.enable";
- public static final String CONFIGURATION = "CONFIGURATION";
- public static final String START_DERBY_ENABLE = "start.derby.server.mode";
- public static final String DERBY_SERVER_MODE_SYS_PROPERTY = "derby.drda.startNetworkServer";
- private static NetworkServerControl server;
- private static JdbcStorage db;
- private static String jdbcURl;
- private static String jdbcDriver;
- private static String jdbcUser;
- private static String jdbcPassword;
-
@PersistenceUnit(unitName = PERSISTENCE_UNIT_NAME)
protected static EntityManagerFactory factory;
@PersistenceContext(unitName = PERSISTENCE_UNIT_NAME)
@@ -102,108 +86,6 @@ public class JPAUtils {
return entityManager;
}
- public static void initializeDB() throws SharingRegistryException {
- jdbcDriver = readServerProperties(SHARING_REG_JDBC_DRIVER);
- jdbcURl = readServerProperties(SHARING_REG_JDBC_URL);
- jdbcUser = readServerProperties(SHARING_REG_JDBC_USER);
- jdbcPassword = readServerProperties(SHARING_REG_JDBC_PWD);
- jdbcURl = jdbcURl + "?" + "user=" + jdbcUser + "&" + "password=" + jdbcPassword;
-
- if (getDBType(jdbcURl).equals("derby") && isDerbyStartEnabled()) {
- startDerbyInServerMode();
- }
- db = new JdbcStorage(10, 50, jdbcURl, jdbcDriver, true);
-
- Connection conn = null;
- try {
- conn = db.connect();
- if (!DatabaseCreator.isDatabaseStructureCreated(CONFIGURATION, conn)) {
- DatabaseCreator.createRegistryDatabase("database_scripts/sharing-registry", conn);
- logger.info("New Database created for Sharing Catalog !!! ");
- } else {
- logger.info("Database already created for Sharing Catalog !!!");
- }
- } catch (Exception e) {
- logger.error(e.getMessage(), e);
- throw new RuntimeException("Database failure", e);
- } finally {
- db.closeConnection(conn);
- try {
- if(conn != null){
- if (!conn.getAutoCommit()) {
- conn.commit();
- }
- conn.close();
- }
- } catch (SQLException e) {
- logger.error("Error while closing database connection...", e.getMessage(), e);
- }
- }
- }
-
- public static String getDBType(String jdbcUrl){
- try{
- String cleanURI = jdbcUrl.substring(5);
- URI uri = URI.create(cleanURI);
- return uri.getScheme();
- } catch (Exception e) {
- logger.error(e.getMessage(), e);
- return null;
- }
- }
-
- public static boolean isDerbyStartEnabled(){
- try {
- String s = ServerSettings.getSetting(START_DERBY_ENABLE);
- if("true".equals(s)){
- return true;
- }
- } catch (ApplicationSettingsException e) {
- logger.error("Unable to read airavata server properties", e.getMessage(), e);
- return false;
- }
- return false;
- }
-
- public static void startDerbyInServerMode() {
- try {
- System.setProperty(DERBY_SERVER_MODE_SYS_PROPERTY, "true");
- server = new NetworkServerControl(InetAddress.getByName("0.0.0.0"),
- getPort(jdbcURl),
- jdbcUser, jdbcPassword);
- java.io.PrintWriter consoleWriter = new java.io.PrintWriter(System.out, true);
- server.start(consoleWriter);
- } catch (IOException e) {
- logger.error("Unable to start Apache derby in the server mode! Check whether " +
- "specified port is available");
- } catch (Exception e) {
- logger.error("Unable to start Apache derby in the server mode! Check whether " +
- "specified port is available");
- }
- }
-
- public static void stopDerbyInServerMode() {
- System.setProperty(DERBY_SERVER_MODE_SYS_PROPERTY, "false");
- if (server!=null){
- try {
- server.shutdown();
- } catch (Exception e) {
- logger.error("Error when stopping the derby server : "+e.getLocalizedMessage());
- }
- }
- }
-
- public static int getPort(String jdbcURL){
- try{
- String cleanURI = jdbcURL.substring(5);
- URI uri = URI.create(cleanURI);
- return uri.getPort();
- } catch (Exception e) {
- logger.error(e.getMessage(), e);
- return -1;
- }
- }
-
public static String readServerProperties(String propertyName) throws SharingRegistryException {
try {
return ServerSettings.getSetting(propertyName);
@@ -212,4 +94,4 @@ public class JPAUtils {
throw new SharingRegistryException("Unable to read airavata-server.properties...");
}
}
-}
\ No newline at end of file
+}
diff --git a/modules/sharing-registry/sharing-registry-server/src/main/java/org/apache/airavata/sharing/registry/db/utils/JdbcStorage.java b/modules/sharing-registry/sharing-registry-server/src/main/java/org/apache/airavata/sharing/registry/db/utils/JdbcStorage.java
deleted file mode 100644
index f344b63..0000000
--- a/modules/sharing-registry/sharing-registry-server/src/main/java/org/apache/airavata/sharing/registry/db/utils/JdbcStorage.java
+++ /dev/null
@@ -1,174 +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.airavata.sharing.registry.db.utils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.sql.*;
-
-public class JdbcStorage {
- private static Logger log = LoggerFactory.getLogger(JdbcStorage.class);
-
- private ConnectionPool connectionPool;
-
- public JdbcStorage(String jdbcUrl, String jdbcDriver) {
- // default init connection and max connection
- this(3, 50, jdbcUrl, jdbcDriver, true);
- }
-
- public JdbcStorage(int initCon, int maxCon, String url, String driver, boolean enableTransactions) {
- try {
- if (enableTransactions) {
- connectionPool = new ConnectionPool(driver, url, initCon, maxCon, true, false,
- Connection.TRANSACTION_SERIALIZABLE);
- } else {
- connectionPool = new ConnectionPool(driver, url, initCon, maxCon, true);
- }
- } catch (Exception e) {
- throw new RuntimeException("Failed to create database connection pool.", e);
- }
- }
-
- /**
- * Check if this connection pool is auto commit or not
- *
- * @return
- */
- public boolean isAutoCommit() {
- return connectionPool.isAutoCommit();
- }
-
- public void commit(Connection conn) {
- try {
- if (conn != null && !conn.getAutoCommit()) {
- conn.commit();
- }
- } catch (SQLException sqle) {
- log.error("Cannot commit data", sqle);
- }
- }
-
- public void commitAndFree(Connection conn) {
- commit(conn);
- closeConnection(conn);
- }
-
- public void rollback(Connection conn) {
- try {
- if (conn != null && !conn.getAutoCommit()) {
- conn.rollback();
- }
- } catch (SQLException sqle) {
- log.error("Cannot Rollback data", sqle);
- }
- }
-
- public void rollbackAndFree(Connection conn) {
- rollback(conn);
- closeConnection(conn);
- }
-
- public Connection connect() {
-
- Connection conn = null;
- try {
- conn = connectionPool.getConnection();
- } catch (SQLException e) {
- log.error(e.getMessage(), e);
- }
- return conn;
- }
-
- /**
- * This method is provided so that you can have better control over the statement. For example: You can use
- * stmt.setString to convert quotation mark automatically in an UPDATE statement
- *
- * NOTE: Statement is closed after execution
- */
- public int executeUpdateAndClose(PreparedStatement stmt) throws SQLException {
- int rows = 0;
- try {
- rows = stmt.executeUpdate();
- if (rows == 0) {
- log.info("Problem: 0 rows affected by insert/update/delete statement.");
- }
- } finally {
- stmt.close();
- }
- return rows;
- }
-
- public int countRow(String tableName, String columnName) throws SQLException {
- String query = new String("SELECT COUNT(" + columnName + ") FROM " + tableName);
- int count = -1;
- Connection conn = null;
- PreparedStatement stmt = null;
- try {
- conn = connectionPool.getConnection();
- stmt = conn.prepareStatement(query);
- ResultSet rs = stmt.executeQuery();
- rs.next();
- count = rs.getInt(1);
- commit(conn);
- } catch (SQLException sql) {
- rollback(conn);
- throw sql;
- } finally {
- try {
- if (stmt != null && !stmt.isClosed()) {
- stmt.close();
- }
- } finally {
- closeConnection(conn);
- }
- }
- return count;
- }
-
- public void quietlyClose(Connection conn, Statement... stmts) {
- if (stmts != null) {
- for (Statement stmt : stmts) {
- try {
- if (stmt != null && !stmt.isClosed()) {
- stmt.close();
- }
- } catch (SQLException sql) {
- log.error(sql.getMessage(), sql);
- }
- }
- }
- closeConnection(conn);
- }
-
- public void closeConnection(Connection conn) {
- if (conn != null) {
- connectionPool.free(conn);
- }
- }
-
- public void closeAllConnections() {
- if (connectionPool != null)
- connectionPool.dispose();
- }
-
- public void shutdown() throws SQLException {
- connectionPool.shutdown();
- }
-}
\ No newline at end of file
diff --git a/modules/sharing-registry/sharing-registry-server/src/main/java/org/apache/airavata/sharing/registry/db/utils/SharingRegistryDBInitConfig.java b/modules/sharing-registry/sharing-registry-server/src/main/java/org/apache/airavata/sharing/registry/db/utils/SharingRegistryDBInitConfig.java
new file mode 100644
index 0000000..3e94809
--- /dev/null
+++ b/modules/sharing-registry/sharing-registry-server/src/main/java/org/apache/airavata/sharing/registry/db/utils/SharingRegistryDBInitConfig.java
@@ -0,0 +1,46 @@
+/*
+ * 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.airavata.sharing.registry.db.utils;
+
+import org.apache.airavata.common.utils.DBInitConfig;
+import org.apache.airavata.common.utils.JDBCConfig;
+
+public class SharingRegistryDBInitConfig implements DBInitConfig {
+ private String dbInitScriptPrefix = "database_scripts/sharing-registry";
+
+ @Override
+ public JDBCConfig getJDBCConfig() {
+ return new SharingRegistryJDBCConfig();
+ }
+
+ @Override
+ public String getDBInitScriptPrefix() {
+ return this.dbInitScriptPrefix;
+ }
+
+ @Override
+ public String getCheckTableName() {
+ return "CONFIGURATION";
+ }
+
+ public void setDBInitScriptPrefix(String dbInitScriptPrefix) {
+ this.dbInitScriptPrefix = dbInitScriptPrefix;
+ }
+}
diff --git a/modules/sharing-registry/sharing-registry-server/src/main/java/org/apache/airavata/sharing/registry/db/utils/SharingRegistryJDBCConfig.java b/modules/sharing-registry/sharing-registry-server/src/main/java/org/apache/airavata/sharing/registry/db/utils/SharingRegistryJDBCConfig.java
new file mode 100644
index 0000000..78ce7e4
--- /dev/null
+++ b/modules/sharing-registry/sharing-registry-server/src/main/java/org/apache/airavata/sharing/registry/db/utils/SharingRegistryJDBCConfig.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+
+package org.apache.airavata.sharing.registry.db.utils;
+
+import org.apache.airavata.common.exception.ApplicationSettingsException;
+import org.apache.airavata.common.utils.JDBCConfig;
+import org.apache.airavata.common.utils.ServerSettings;
+
+public class SharingRegistryJDBCConfig implements JDBCConfig {
+
+ public static final String SHARING_REG_JDBC_DRIVER = "sharingcatalog.jdbc.driver";
+ public static final String SHARING_REG_JDBC_URL = "sharingcatalog.jdbc.url";
+ public static final String SHARING_REG_JDBC_USER = "sharingcatalog.jdbc.user";
+ public static final String SHARING_REG_JDBC_PWD = "sharingcatalog.jdbc.password";
+
+ @Override
+ public String getURL() {
+ return readServerProperties(SHARING_REG_JDBC_URL);
+ }
+
+ @Override
+ public String getDriver() {
+ return readServerProperties(SHARING_REG_JDBC_DRIVER);
+ }
+
+ @Override
+ public String getUser() {
+ return readServerProperties(SHARING_REG_JDBC_USER);
+ }
+
+ @Override
+ public String getPassword() {
+ return readServerProperties(SHARING_REG_JDBC_PWD);
+ }
+
+ private String readServerProperties(String propertyName) {
+ try {
+ return ServerSettings.getSetting(propertyName);
+ } catch (ApplicationSettingsException e) {
+ throw new RuntimeException("Unable to read airavata-server.properties...", e);
+ }
+ }
+}
diff --git a/modules/sharing-registry/sharing-registry-server/src/main/java/org/apache/airavata/sharing/registry/server/SharingRegistryServer.java b/modules/sharing-registry/sharing-registry-server/src/main/java/org/apache/airavata/sharing/registry/server/SharingRegistryServer.java
index ebca1cc..e24196b 100644
--- a/modules/sharing-registry/sharing-registry-server/src/main/java/org/apache/airavata/sharing/registry/server/SharingRegistryServer.java
+++ b/modules/sharing-registry/sharing-registry-server/src/main/java/org/apache/airavata/sharing/registry/server/SharingRegistryServer.java
@@ -22,6 +22,7 @@ package org.apache.airavata.sharing.registry.server;
import org.apache.airavata.common.exception.AiravataException;
import org.apache.airavata.common.utils.IServer;
import org.apache.airavata.common.utils.ServerSettings;
+import org.apache.airavata.sharing.registry.db.utils.SharingRegistryDBInitConfig;
import org.apache.airavata.sharing.registry.messaging.SharingServiceDBEventMessagingFactory;
import org.apache.airavata.sharing.registry.models.SharingRegistryException;
import org.apache.airavata.sharing.registry.service.cpi.SharingRegistryService;
@@ -49,6 +50,7 @@ public class SharingRegistryServer implements IServer {
private IServer.ServerStatus status;
private TServer server;
+ private boolean testMode = false;
public SharingRegistryServer() {
setStatus(IServer.ServerStatus.STOPPED);
@@ -71,7 +73,8 @@ public class SharingRegistryServer implements IServer {
final int serverPort = Integer.parseInt(ServerSettings.getSetting(SHARING_REG_SERVER_PORT));
final String serverHost = ServerSettings.getSetting(SHARING_REG_SERVER_HOST);
- SharingRegistryService.Processor processor = new SharingRegistryService.Processor(new SharingRegistryServerHandler());
+ SharingRegistryService.Processor processor = new SharingRegistryService.Processor(
+ new SharingRegistryServerHandler(createSharingRegistryDBInitConfig()));
TServerTransport serverTransport;
@@ -172,4 +175,20 @@ public class SharingRegistryServer implements IServer {
public void setServer(TServer server) {
this.server = server;
}
-}
\ No newline at end of file
+
+ public boolean isTestMode() {
+ return testMode;
+ }
+
+ public void setTestMode(boolean testMode) {
+ this.testMode = testMode;
+ }
+
+ private SharingRegistryDBInitConfig createSharingRegistryDBInitConfig() {
+ SharingRegistryDBInitConfig sharingRegistryDBInitConfig = new SharingRegistryDBInitConfig();
+ if (this.testMode) {
+ sharingRegistryDBInitConfig.setDBInitScriptPrefix("sharing-registry");
+ }
+ return sharingRegistryDBInitConfig;
+ }
+}
diff --git a/modules/sharing-registry/sharing-registry-server/src/main/java/org/apache/airavata/sharing/registry/server/SharingRegistryServerHandler.java b/modules/sharing-registry/sharing-registry-server/src/main/java/org/apache/airavata/sharing/registry/server/SharingRegistryServerHandler.java
index 7f5d405..20ba2de 100644
--- a/modules/sharing-registry/sharing-registry-server/src/main/java/org/apache/airavata/sharing/registry/server/SharingRegistryServerHandler.java
+++ b/modules/sharing-registry/sharing-registry-server/src/main/java/org/apache/airavata/sharing/registry/server/SharingRegistryServerHandler.java
@@ -20,10 +20,11 @@
package org.apache.airavata.sharing.registry.server;
import org.apache.airavata.common.exception.ApplicationSettingsException;
+import org.apache.airavata.common.utils.DBInitializer;
import org.apache.airavata.sharing.registry.db.entities.*;
import org.apache.airavata.sharing.registry.db.repositories.*;
import org.apache.airavata.sharing.registry.db.utils.DBConstants;
-import org.apache.airavata.sharing.registry.db.utils.JPAUtils;
+import org.apache.airavata.sharing.registry.db.utils.SharingRegistryDBInitConfig;
import org.apache.airavata.sharing.registry.models.*;
import org.apache.airavata.sharing.registry.service.cpi.SharingRegistryService;
import org.apache.commons.lang.exception.ExceptionUtils;
@@ -41,7 +42,11 @@ public class SharingRegistryServerHandler implements SharingRegistryService.Ifac
public static String OWNER_PERMISSION_NAME = "OWNER";
public SharingRegistryServerHandler() throws ApplicationSettingsException, TException {
- JPAUtils.initializeDB();
+ this(new SharingRegistryDBInitConfig());
+ }
+
+ public SharingRegistryServerHandler(SharingRegistryDBInitConfig sharingRegistryDBInitConfig) throws ApplicationSettingsException, TException {
+ DBInitializer.initializeDB(sharingRegistryDBInitConfig);
}
/**
@@ -1216,4 +1221,4 @@ public class SharingRegistryServerHandler implements SharingRegistryService.Ifac
}
return hashtable;
}
-}
\ No newline at end of file
+}
diff --git a/modules/sharing-registry/sharing-registry-server/src/test/java/org/apache/airavata/sharing/registry/SharingRegistryServerHandlerTest.java b/modules/sharing-registry/sharing-registry-server/src/test/java/org/apache/airavata/sharing/registry/SharingRegistryServerHandlerTest.java
index 72e1f5a..4804a39 100644
--- a/modules/sharing-registry/sharing-registry-server/src/test/java/org/apache/airavata/sharing/registry/SharingRegistryServerHandlerTest.java
+++ b/modules/sharing-registry/sharing-registry-server/src/test/java/org/apache/airavata/sharing/registry/SharingRegistryServerHandlerTest.java
@@ -21,30 +21,24 @@ package org.apache.airavata.sharing.registry;
import org.junit.Assert;
import org.apache.airavata.common.exception.ApplicationSettingsException;
+import org.apache.airavata.sharing.registry.db.utils.SharingRegistryDBInitConfig;
import org.apache.airavata.sharing.registry.models.*;
import org.apache.airavata.sharing.registry.server.SharingRegistryServerHandler;
-import org.apache.airavata.sharing.registry.util.Initialize;
import org.apache.thrift.TException;
-import org.junit.BeforeClass;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.sql.SQLException;
import java.util.ArrayList;
import java.util.Arrays;
public class SharingRegistryServerHandlerTest {
private final static Logger logger = LoggerFactory.getLogger(SharingRegistryServerHandlerTest.class);
- @BeforeClass
- public static void setup() throws SharingRegistryException, SQLException {
- Initialize initialize = new Initialize("sharing-registry-derby.sql");
- initialize.initializeDB();
- }
-
@Test
public void test() throws TException, ApplicationSettingsException {
+ SharingRegistryDBInitConfig sharingRegistryDBInitConfig = new SharingRegistryDBInitConfig();
+ sharingRegistryDBInitConfig.setDBInitScriptPrefix("sharing-registry");
SharingRegistryServerHandler sharingRegistryServerHandler = new SharingRegistryServerHandler();
//Creating domain
@@ -345,4 +339,4 @@ public class SharingRegistryServerHandlerTest {
Assert.assertTrue(sharingRegistryServerHandler.getListOfSharedUsers(domainId, entityId1, domainId + ":OWNER").size()==1);
}
-}
\ No newline at end of file
+}
diff --git a/modules/sharing-registry/sharing-registry-server/src/test/java/org/apache/airavata/sharing/registry/SharingRegistryServiceTest.java b/modules/sharing-registry/sharing-registry-server/src/test/java/org/apache/airavata/sharing/registry/SharingRegistryServiceTest.java
index 72fd828..4bc78f1 100644
--- a/modules/sharing-registry/sharing-registry-server/src/test/java/org/apache/airavata/sharing/registry/SharingRegistryServiceTest.java
+++ b/modules/sharing-registry/sharing-registry-server/src/test/java/org/apache/airavata/sharing/registry/SharingRegistryServiceTest.java
@@ -22,7 +22,7 @@ package org.apache.airavata.sharing.registry;
import org.apache.airavata.common.exception.ApplicationSettingsException;
import org.apache.airavata.common.utils.ServerSettings;
import org.apache.airavata.sharing.registry.models.*;
-import org.apache.airavata.sharing.registry.server.ServerMain;
+import org.apache.airavata.sharing.registry.server.SharingRegistryServer;
import org.apache.airavata.sharing.registry.service.cpi.SharingRegistryService;
import org.apache.thrift.TException;
import org.apache.thrift.protocol.TBinaryProtocol;
@@ -43,9 +43,10 @@ public class SharingRegistryServiceTest {
private final static Logger logger = LoggerFactory.getLogger(SharingRegistryServiceTest.class);
@BeforeClass
- public static void setUp() throws InterruptedException {
- ServerMain serverMain = new ServerMain();
- serverMain.main(new String[]{});
+ public static void setUp() throws Exception {
+ SharingRegistryServer server = new SharingRegistryServer();
+ server.setTestMode(true);
+ server.start();
Thread.sleep(1000 * 2);
}
@@ -415,4 +416,4 @@ public class SharingRegistryServiceTest {
filters.add(searchCriteria);
Assert.assertTrue(sharingServiceClient.searchEntities(domainId, "test-user-2", filters, 0, -1).size() == 0);
}
-}
\ No newline at end of file
+}
diff --git a/modules/sharing-registry/sharing-registry-server/src/test/java/org/apache/airavata/sharing/registry/util/Initialize.java b/modules/sharing-registry/sharing-registry-server/src/test/java/org/apache/airavata/sharing/registry/util/Initialize.java
deleted file mode 100644
index 2c7ef46..0000000
--- a/modules/sharing-registry/sharing-registry-server/src/test/java/org/apache/airavata/sharing/registry/util/Initialize.java
+++ /dev/null
@@ -1,296 +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.airavata.sharing.registry.util;
-
-import org.apache.airavata.common.exception.ApplicationSettingsException;
-import org.apache.airavata.common.utils.ServerSettings;
-import org.apache.airavata.sharing.registry.db.utils.JPAUtils;
-import org.apache.derby.drda.NetworkServerControl;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.BufferedReader;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InputStreamReader;
-import java.net.InetAddress;
-import java.net.URI;
-import java.sql.*;
-import java.util.StringTokenizer;
-
-public class Initialize {
- private static final Logger logger = LoggerFactory.getLogger(Initialize.class);
- public static final String DERBY_SERVER_MODE_SYS_PROPERTY = "derby.drda.startNetworkServer";
- public String scriptName ;
- private NetworkServerControl server;
- private static final String delimiter = ";";
- public static final String PERSISTANT_DATA = "Configuration";
-
- public Initialize(String scriptName) {
- this.scriptName = scriptName;
- }
-
- public static boolean checkStringBufferEndsWith(StringBuffer buffer, String suffix) {
- if (suffix.length() > buffer.length()) {
- return false;
- }
- // this loop is done on purpose to avoid memory allocation performance
- // problems on various JDKs
- // StringBuffer.lastIndexOf() was introduced in jdk 1.4 and
- // implementation is ok though does allocation/copying
- // StringBuffer.toString().endsWith() does massive memory
- // allocation/copying on JDK 1.5
- // See http://issues.apache.org/bugzilla/show_bug.cgi?id=37169
- int endIndex = suffix.length() - 1;
- int bufferIndex = buffer.length() - 1;
- while (endIndex >= 0) {
- if (buffer.charAt(bufferIndex) != suffix.charAt(endIndex)) {
- return false;
- }
- bufferIndex--;
- endIndex--;
- }
- return true;
- }
-
- private static boolean isServerStarted(NetworkServerControl server, int ntries)
- {
- for (int i = 1; i <= ntries; i ++)
- {
- try {
- Thread.sleep(500);
- server.ping();
- return true;
- }
- catch (Exception e) {
- if (i == ntries)
- return false;
- }
- }
- return false;
- }
-
- public void initializeDB() throws SQLException{
- String jdbcUrl = null;
- String jdbcUser = null;
- String jdbcPassword = null;
- try{
- jdbcUrl = ServerSettings.getSetting("sharingcatalog.jdbc.url");
- jdbcUser = ServerSettings.getSetting("sharingcatalog.jdbc.user");
- jdbcPassword = ServerSettings.getSetting("sharingcatalog.jdbc.password");
- jdbcUrl = jdbcUrl + "?" + "user=" + jdbcUser + "&" + "password=" + jdbcPassword;
- } catch (ApplicationSettingsException e) {
- logger.error("Unable to read properties", e);
- }
- startDerbyInServerMode();
- if(!isServerStarted(server, 20)){
- throw new RuntimeException("Derby server cound not started within five seconds...");
- }
-
- Connection conn = null;
- try {
- Class.forName(JPAUtils.readServerProperties(JPAUtils.SHARING_REG_JDBC_DRIVER)).newInstance();
- conn = DriverManager.getConnection(jdbcUrl, jdbcUser, jdbcPassword);
- if (!isDatabaseStructureCreated(PERSISTANT_DATA, conn)) {
- executeSQLScript(conn);
- logger.info("New Database created for Registry");
- } else {
- logger.debug("Database already created for Registry!");
- }
- } catch (Exception e) {
- logger.error(e.getMessage(), e);
- throw new RuntimeException("Database failure", e);
- } finally {
- try {
- if (conn != null){
- if (!conn.getAutoCommit()) {
- conn.commit();
- }
- conn.close();
- }
- } catch (SQLException e) {
- logger.error(e.getMessage(), e);
- }
- }
- }
-
- public static boolean isDatabaseStructureCreated(String tableName, Connection conn) {
- try {
- System.out.println("Running a query to test the database tables existence.");
- // check whether the tables are already created with a query
- Statement statement = null;
- try {
- statement = conn.createStatement();
- ResultSet rs = statement.executeQuery("select * from " + tableName);
- if (rs != null) {
- rs.close();
- }
- } finally {
- try {
- if (statement != null) {
- statement.close();
- }
- } catch (SQLException e) {
- return false;
- }
- }
- } catch (SQLException e) {
- return false;
- }
-
- return true;
- }
-
- private void executeSQLScript(Connection conn) throws Exception {
- StringBuffer sql = new StringBuffer();
- BufferedReader reader = null;
- try{
-
- InputStream inputStream = this.getClass().getClassLoader().getResourceAsStream(scriptName);
- reader = new BufferedReader(new InputStreamReader(inputStream));
- String line;
- while ((line = reader.readLine()) != null) {
- line = line.trim();
- if (line.startsWith("//")) {
- continue;
- }
- if (line.startsWith("--")) {
- continue;
- }
- StringTokenizer st = new StringTokenizer(line);
- if (st.hasMoreTokens()) {
- String token = st.nextToken();
- if ("REM".equalsIgnoreCase(token)) {
- continue;
- }
- }
- sql.append(" ").append(line);
-
- // SQL defines "--" as a comment to EOL
- // and in Oracle it may contain a hint
- // so we cannot just remove it, instead we must end it
- if (line.indexOf("--") >= 0) {
- sql.append("\n");
- }
- if ((checkStringBufferEndsWith(sql, delimiter))) {
- executeSQL(sql.substring(0, sql.length() - delimiter.length()), conn);
- sql.replace(0, sql.length(), "");
- }
- }
- // Catch any statements not followed by ;
- if (sql.length() > 0) {
- executeSQL(sql.toString(), conn);
- }
- }catch (IOException e){
- logger.error("Error occurred while executing SQL script for creating Airavata database", e);
- throw new Exception("Error occurred while executing SQL script for creating Airavata database", e);
- }finally {
- if (reader != null) {
- reader.close();
- }
-
- }
-
- }
-
- private static void executeSQL(String sql, Connection conn) throws Exception {
- // Check and ignore empty statements
- if ("".equals(sql.trim())) {
- return;
- }
-
- Statement statement = null;
- try {
- logger.debug("SQL : " + sql);
-
- boolean ret;
- int updateCount = 0, updateCountTotal = 0;
- statement = conn.createStatement();
- ret = statement.execute(sql);
- updateCount = statement.getUpdateCount();
- do {
- if (!ret) {
- if (updateCount != -1) {
- updateCountTotal += updateCount;
- }
- }
- ret = statement.getMoreResults();
- if (ret) {
- updateCount = statement.getUpdateCount();
- }
- } while (ret);
-
- logger.debug(sql + " : " + updateCountTotal + " rows affected");
-
- SQLWarning warning = conn.getWarnings();
- while (warning != null) {
- logger.warn(warning + " sql warning");
- warning = warning.getNextWarning();
- }
- conn.clearWarnings();
- } catch (SQLException e) {
- if (e.getSQLState().equals("X0Y32")) {
- // eliminating the table already exception for the derby
- // database
- logger.info("Table Already Exists", e);
- } else {
- throw new Exception("Error occurred while executing : " + sql, e);
- }
- } finally {
- if (statement != null) {
- try {
- statement.close();
- } catch (SQLException e) {
- logger.error("Error occurred while closing result set.", e);
- }
- }
- }
- }
-
- private void startDerbyInServerMode() {
- try {
- System.setProperty(DERBY_SERVER_MODE_SYS_PROPERTY, "true");
- String jdbcURL = JPAUtils.readServerProperties(JPAUtils.SHARING_REG_JDBC_URL);
- String cleanURI = jdbcURL.substring(5);
- URI uri = URI.create(cleanURI);
- server = new NetworkServerControl(InetAddress.getByName(uri.getHost()),
- 20000,
- JPAUtils.readServerProperties(JPAUtils.SHARING_REG_JDBC_USER), JPAUtils.readServerProperties(JPAUtils.SHARING_REG_JDBC_USER));
- java.io.PrintWriter consoleWriter = new java.io.PrintWriter(System.out, true);
- server.start(consoleWriter);
- } catch (IOException e) {
- logger.error("Unable to start Apache derby in the server mode! Check whether " +
- "specified port is available");
- } catch (Exception e) {
- logger.error("Unable to start Apache derby in the server mode! Check whether " +
- "specified port is available");
- }
-
- }
-
- public void stopDerbyServer() throws SQLException{
- try {
- server.shutdown();
- } catch (Exception e) {
- logger.error(e.getMessage(), e);
- throw new SQLException("Error while stopping derby server", e);
- }
- }
-}
|