aboutsummaryrefslogtreecommitdiffstats
diff options
context:
space:
mode:
-rwxr-xr-xmdbc-server/src/main/java/org/onap/music/mdbc/MdbcConnection.java24
-rwxr-xr-xmdbc-server/src/main/java/org/onap/music/mdbc/MdbcServer.java1
-rwxr-xr-xmdbc-server/src/main/java/org/onap/music/mdbc/mixins/DBInterface.java4
-rwxr-xr-xmdbc-server/src/main/java/org/onap/music/mdbc/mixins/MusicInterface.java15
-rw-r--r--mdbc-server/src/main/java/org/onap/music/mdbc/mixins/MusicMixin.java70
-rwxr-xr-xmdbc-server/src/main/java/org/onap/music/mdbc/mixins/MySQLMixin.java15
-rwxr-xr-xmdbc-server/src/main/java/org/onap/music/mdbc/mixins/PostgresMixin.java8
-rw-r--r--mdbc-server/src/main/java/org/onap/music/mdbc/ownership/OwnershipAndCheckpoint.java21
-rw-r--r--mdbc-server/src/main/java/org/onap/music/mdbc/tables/MusicTxDigestDaemon.java3
-rwxr-xr-xmdbc-server/src/main/resources/mdbc.properties6
-rw-r--r--mdbc-server/src/test/java/org/onap/music/mdbc/MdbcTestUtils.java2
-rw-r--r--mdbc-server/src/test/java/org/onap/music/mdbc/mixins/MySQLMixinTest.java2
-rw-r--r--mdbc-server/src/test/java/org/onap/music/mdbc/mixins/PostgresMixinTest.java2
13 files changed, 128 insertions, 45 deletions
diff --git a/mdbc-server/src/main/java/org/onap/music/mdbc/MdbcConnection.java b/mdbc-server/src/main/java/org/onap/music/mdbc/MdbcConnection.java
index 2294673..6f097dd 100755
--- a/mdbc-server/src/main/java/org/onap/music/mdbc/MdbcConnection.java
+++ b/mdbc-server/src/main/java/org/onap/music/mdbc/MdbcConnection.java
@@ -82,9 +82,13 @@ public class MdbcConnection implements Connection {
private final TxCommitProgress progressKeeper;
private final DBInterface dbi;
private final StagingTable transactionDigest;
+ /** Set of tables in db */
private final Set<String> table_set;
private final StateManager statemanager;
+ /** partition owned for this transaction */
private DatabasePartition partition;
+ /** ranges needed for this transaction */
+ private Set<Range> rangesUsed;
public MdbcConnection(String id, String url, Connection c, Properties info, MusicInterface mi,
TxCommitProgress progressKeeper, DatabasePartition partition, StateManager statemanager) throws MDBCServiceException {
@@ -187,6 +191,13 @@ public class MdbcConnection implements Connection {
dbi.preCommitHook();
try {
+ partition = mi.splitPartitionIfNecessary(partition, rangesUsed);
+ } catch (MDBCServiceException e) {
+ logger.warn(EELFLoggerDelegate.errorLogger, "Failure to split partition, trying to continue",
+ AppMessages.UNKNOWNERROR, ErrorTypes.UNKNOWN, ErrorSeverity.FATAL);
+ }
+
+ try {
logger.debug(EELFLoggerDelegate.applicationLogger, " commit ");
// transaction was committed -- add all the updates into the REDO-Log in MUSIC
mi.commitLog(partition, statemanager.getEventualRanges(), transactionDigest, id, progressKeeper);
@@ -519,16 +530,15 @@ public class MdbcConnection implements Connection {
//Check ownership of keys
String defaultSchema = dbi.getSchema();
Set<Range> queryTables = MDBCUtils.getTables(defaultSchema, tableToQueryType);
- if (this.partition!=null) {
- Set<Range> snapshot = this.partition.getSnapshot();
- if(snapshot!=null){
- queryTables.addAll(snapshot);
- }
+ if (this.rangesUsed==null) {
+ rangesUsed = queryTables;
+ } else {
+ rangesUsed.addAll(queryTables);
}
// filter out ranges that fall under Eventually consistent
// category as these tables do not need ownership
- Set<Range> scQueryTables = filterEveTables(queryTables);
- DatabasePartition tempPartition = own(scQueryTables, MDBCUtils.getOperationType(tableToQueryType));
+ Set<Range> scRanges = filterEveTables(rangesUsed);
+ DatabasePartition tempPartition = own(scRanges, MDBCUtils.getOperationType(tableToQueryType));
if(tempPartition!=null && tempPartition != partition) {
this.partition.updateDatabasePartition(tempPartition);
statemanager.getOwnAndCheck().reloadAlreadyApplied(this.partition);
diff --git a/mdbc-server/src/main/java/org/onap/music/mdbc/MdbcServer.java b/mdbc-server/src/main/java/org/onap/music/mdbc/MdbcServer.java
index 500ed81..246044b 100755
--- a/mdbc-server/src/main/java/org/onap/music/mdbc/MdbcServer.java
+++ b/mdbc-server/src/main/java/org/onap/music/mdbc/MdbcServer.java
@@ -101,6 +101,7 @@ public class MdbcServer {
// Then start it
server.start();
+ System.out.println("Started Avatica server on port " + server.getPort());
logger.info("Started Avatica server on port {} with serialization {}", server.getPort(),
serialization);
} catch (Exception e) {
diff --git a/mdbc-server/src/main/java/org/onap/music/mdbc/mixins/DBInterface.java b/mdbc-server/src/main/java/org/onap/music/mdbc/mixins/DBInterface.java
index 745307c..15dd456 100755
--- a/mdbc-server/src/main/java/org/onap/music/mdbc/mixins/DBInterface.java
+++ b/mdbc-server/src/main/java/org/onap/music/mdbc/mixins/DBInterface.java
@@ -134,13 +134,13 @@ public interface DBInterface {
* @throws SQLException if replay cannot occur correctly
* @throws MDBCServiceException
*/
- void replayTransaction(StagingTable digest, Set<Range> ranges) throws SQLException, MDBCServiceException;
+ void replayTransaction(StagingTable digest) throws SQLException, MDBCServiceException;
void disableForeignKeyChecks() throws SQLException;
void enableForeignKeyChecks() throws SQLException;
- void applyTxDigest(StagingTable txDigest, Set<Range> ranges) throws SQLException, MDBCServiceException;
+ void applyTxDigest(StagingTable txDigest) throws SQLException, MDBCServiceException;
Connection getSQLConnection();
diff --git a/mdbc-server/src/main/java/org/onap/music/mdbc/mixins/MusicInterface.java b/mdbc-server/src/main/java/org/onap/music/mdbc/mixins/MusicInterface.java
index 637cb15..3afc726 100755
--- a/mdbc-server/src/main/java/org/onap/music/mdbc/mixins/MusicInterface.java
+++ b/mdbc-server/src/main/java/org/onap/music/mdbc/mixins/MusicInterface.java
@@ -181,6 +181,7 @@ public interface MusicInterface {
/**
* Commits the corresponding REDO-log into MUSIC
+ * Transaction is committed -- add all the updates into the REDO-Log in MUSIC
*
* @param partition information related to ownership of partitions, used to verify ownership when commiting the Tx
* @param eventualRanges
@@ -337,7 +338,19 @@ public interface MusicInterface {
*/
OwnershipReturn mergeLatestRowsIfNecessary(Dag currentlyOwned, Map<UUID, LockResult> locksForOwnership, UUID ownershipId)
throws MDBCServiceException;
-
+
+ /**
+ * If this connection is using fewer ranges than what is owned in the current partition, split
+ * the partition to avoid a universal partition being passed around.
+ *
+ * This will follow "most recently used" policy
+ * @param partition2 partition that this transaction currently owns
+ * @param rangesUsed set of ranges that is the minimal required for this transaction
+ * @throws MDBCServiceException
+ */
+ public DatabasePartition splitPartitionIfNecessary(DatabasePartition partition, Set<Range> rangesUsed)
+ throws MDBCServiceException;
+
/**
* Create ranges in MRI table, if not already present
* @param range to add into mri table
diff --git a/mdbc-server/src/main/java/org/onap/music/mdbc/mixins/MusicMixin.java b/mdbc-server/src/main/java/org/onap/music/mdbc/mixins/MusicMixin.java
index 5581573..e87f7e4 100644
--- a/mdbc-server/src/main/java/org/onap/music/mdbc/mixins/MusicMixin.java
+++ b/mdbc-server/src/main/java/org/onap/music/mdbc/mixins/MusicMixin.java
@@ -104,6 +104,8 @@ public class MusicMixin implements MusicInterface {
public static final String KEY_TIMEOUT = "mdbc_timeout";
/** The property name to use to provide a flag indicating if compression is required */
public static final String KEY_COMPRESSION = "mdbc_compression";
+ /** The property name to use to provide a flag indicating if mri row splits is allowable */
+ public static final String KEY_SPLIT = "partition_splitting";
/** Namespace for the tables in MUSIC (Cassandra) */
public static final String DEFAULT_MUSIC_NAMESPACE = "namespace";
/** The default property value to use for the Cassandra IP address. */
@@ -197,9 +199,9 @@ public class MusicMixin implements MusicInterface {
private Set<String> in_progress = Collections.synchronizedSet(new HashSet<String>());
private StateManager stateManager;
private boolean useCompression;
+ private boolean splitAllowed;
public MusicMixin() {
-
//this.logger = null;
this.musicAddress = null;
this.music_ns = null;
@@ -237,6 +239,9 @@ public class MusicMixin implements MusicInterface {
String s = info.getProperty(KEY_MUSIC_RFACTOR);
this.music_rfactor = (s == null) ? DEFAULT_MUSIC_RFACTOR : Integer.parseInt(s);
+ String split = info.getProperty(KEY_SPLIT);
+ this.splitAllowed = (split == null) ? true: Boolean.parseBoolean(split);
+
initializeMetricTables();
commitExecutorThreads = Executors.newFixedThreadPool(4);
}
@@ -1263,15 +1268,15 @@ public class MusicMixin implements MusicInterface {
return lockId;
}
- protected void changeIsLatestToMRI(MusicRangeInformationRow row, boolean isLatest, LockResult lock) throws MDBCServiceException{
+ protected void changeIsLatestToMRI(UUID mrirow, boolean isLatest, String lockref) throws MDBCServiceException{
- if(lock == null)
+ if(lockref == null)
return;
- PreparedQueryObject appendQuery = createChangeIsLatestToMriQuery(musicRangeInformationTableName, row.getPartitionIndex(),
+ PreparedQueryObject appendQuery = createChangeIsLatestToMriQuery(musicRangeInformationTableName, mrirow,
musicTxDigestTableName, isLatest);
- ReturnType returnType = MusicCore.criticalPut(music_ns, musicRangeInformationTableName, row.getPartitionIndex().toString(),
+ ReturnType returnType = MusicCore.criticalPut(music_ns, musicRangeInformationTableName, mrirow.toString(),
appendQuery,
- lock.getLockId()
+ lockref
, null);
if(returnType.getResult().compareTo(ResultType.SUCCESS) != 0 ){
logger.error(EELFLoggerDelegate.errorLogger, "Error when executing change isLatest operation with return type: "+returnType.getMessage());
@@ -2140,7 +2145,10 @@ public class MusicMixin implements MusicInterface {
List<MusicRangeInformationRow> returnInfo = new ArrayList<>();
List<DagNode> toDisable = latestDag.getOldestDoubles();
for(DagNode node : toDisable){
- changeIsLatestToMRI(node.getRow(),false,locks.get(node.getId()));
+ LockResult lockToDisable = locks.get(node.getId());
+ if (lockToDisable!=null) {
+ changeIsLatestToMRI(node.getRow().getPartitionIndex(),false,lockToDisable.getLockId());
+ }
latestDag.setIsLatest(node.getId(),false);
returnInfo.add(node.getRow());
}
@@ -2185,6 +2193,54 @@ public class MusicMixin implements MusicInterface {
return new OwnershipReturn(ownershipId, createdRow.getDBPartition().getLockId(), createdRow.getPartitionIndex(),
createdRow.getDBPartition().getSnapshot(), currentlyOwned);
}
+
+
+ @Override
+ public DatabasePartition splitPartitionIfNecessary(DatabasePartition partition, Set<Range> rangesUsed)
+ throws MDBCServiceException {
+ if (!this.splitAllowed) {
+ return partition;
+ }
+ Set<Range> rangesOwned = partition.getSnapshot();
+ if (rangesOwned==null || rangesUsed==null) {
+ return partition;
+ }
+ if (!rangesOwned.containsAll(rangesUsed)) {
+ throw new MDBCServiceException("Transaction was unable to acquire all necessary ranges.");
+ }
+
+ if (rangesUsed.containsAll(rangesOwned)) {
+ //using all ranges in this partition
+ return partition;
+ }
+
+ //split partition
+ logger.info(EELFLoggerDelegate.applicationLogger, "Full partition not being used need (" + rangesUsed
+ +") and own (" + rangesOwned + ", splitting the partition");
+ Set<UUID> prevPartitions = new HashSet<>();
+ prevPartitions.add(partition.getMRIIndex());
+ MusicRangeInformationRow usedRow = createAndAssignLock(rangesUsed, prevPartitions);
+ rangesOwned.removeAll(rangesUsed);
+ Set<Range> rangesNotUsed = rangesOwned;
+ MusicRangeInformationRow unusedRow = createAndAssignLock(rangesNotUsed, prevPartitions);
+
+ changeIsLatestToMRI(partition.getMRIIndex(), false, partition.getLockId());
+
+ Map<Range, Pair<MriReference, Integer>> alreadyApplied = stateManager.getOwnAndCheck().getAlreadyApplied();
+ for (Range range: rangesUsed) {
+ alreadyApplied.put(range, Pair.of(new MriReference(usedRow.getPartitionIndex()), -1));
+ }
+ for (Range range: rangesNotUsed) {
+ alreadyApplied.put(range, Pair.of(new MriReference(unusedRow.getPartitionIndex()), -1));
+ }
+
+ //release/update old partition info
+ relinquish(unusedRow.getDBPartition());
+ relinquish(partition);
+
+ return usedRow.getDBPartition();
+ }
+
private MusicRangeInformationRow createAndAssignLock(Set<Range> ranges, Set<UUID> prevPartitions) throws MDBCServiceException {
UUID newUUID = MDBCUtils.generateTimebasedUniqueKey();
diff --git a/mdbc-server/src/main/java/org/onap/music/mdbc/mixins/MySQLMixin.java b/mdbc-server/src/main/java/org/onap/music/mdbc/mixins/MySQLMixin.java
index 8cab635..3af6f0f 100755
--- a/mdbc-server/src/main/java/org/onap/music/mdbc/mixins/MySQLMixin.java
+++ b/mdbc-server/src/main/java/org/onap/music/mdbc/mixins/MySQLMixin.java
@@ -185,7 +185,7 @@ public class MySQLMixin implements DBInterface {
String dbname = "mdbc"; // default name
try {
Statement stmt = conn.createStatement();
- ResultSet rs = stmt.executeQuery("SELECT DATABASE() AS DB");
+ ResultSet rs = stmt.executeQuery("SELECT UPPER(DATABASE()) AS DB");
if (rs.next()) {
dbname = rs.getString("DB");
}
@@ -212,7 +212,7 @@ public class MySQLMixin implements DBInterface {
public Set<String> getSQLTableSet() {
Set<String> set = new TreeSet<String>();
String sql =
- "SELECT CONCAT(TABLE_SCHEMA, '.', TABLE_NAME) as TABLE_NAME FROM INFORMATION_SCHEMA.TABLES WHERE TABLE_SCHEMA=DATABASE() AND TABLE_TYPE='BASE TABLE'";
+ "SELECT CONCAT(UPPER(TABLE_SCHEMA), '.', UPPER(TABLE_NAME)) as TABLE_NAME FROM INFORMATION_SCHEMA.TABLES WHERE TABLE_SCHEMA=DATABASE() AND TABLE_TYPE='BASE TABLE';";
try {
Statement stmt = jdbcConn.createStatement();
ResultSet rs = stmt.executeQuery(sql);
@@ -232,7 +232,7 @@ public class MySQLMixin implements DBInterface {
public Set<Range> getSQLRangeSet() {
Set<String> set = new TreeSet<String>();
String sql =
- "SELECT CONCAT(TABLE_SCHEMA, '.', TABLE_NAME) as TABLE_NAME FROM INFORMATION_SCHEMA.TABLES WHERE TABLE_SCHEMA=DATABASE() AND TABLE_TYPE='BASE TABLE'";
+ "SELECT CONCAT(UPPER(TABLE_SCHEMA), '.', UPPER(TABLE_NAME)) as TABLE_NAME FROM INFORMATION_SCHEMA.TABLES WHERE TABLE_SCHEMA=DATABASE() AND TABLE_TYPE='BASE TABLE';";
try {
Statement stmt = jdbcConn.createStatement();
ResultSet rs = stmt.executeQuery(sql);
@@ -891,7 +891,7 @@ public class MySQLMixin implements DBInterface {
* @param transaction - base 64 encoded, serialized digest
* @throws MDBCServiceException
*/
- public void replayTransaction(StagingTable transaction, Set<Range> ranges)
+ public void replayTransaction(StagingTable transaction)
throws SQLException, MDBCServiceException {
boolean autocommit = jdbcConn.getAutoCommit();
jdbcConn.setAutoCommit(false);
@@ -899,7 +899,6 @@ public class MySQLMixin implements DBInterface {
ArrayList<Operation> opList = transaction.getOperationList();
for (Operation op : opList) {
- if (Range.overlaps(ranges, op.getTable())) {
try {
replayOperationIntoDB(jdbcStmt, op);
} catch (SQLException | MDBCServiceException e) {
@@ -909,7 +908,6 @@ public class MySQLMixin implements DBInterface {
jdbcConn.rollback();
throw e;
}
- }
}
clearReplayedOperations(jdbcStmt);
@@ -934,8 +932,8 @@ public class MySQLMixin implements DBInterface {
}
@Override
- public void applyTxDigest(StagingTable txDigest, Set<Range> ranges) throws SQLException, MDBCServiceException {
- replayTransaction(txDigest, ranges);
+ public void applyTxDigest(StagingTable txDigest) throws SQLException, MDBCServiceException {
+ replayTransaction(txDigest);
}
/**
@@ -1183,5 +1181,4 @@ public class MySQLMixin implements DBInterface {
logger.error(EELFLoggerDelegate.errorLogger, "initTables: problem creating th mdbc tables!");
}
}
-
} \ No newline at end of file
diff --git a/mdbc-server/src/main/java/org/onap/music/mdbc/mixins/PostgresMixin.java b/mdbc-server/src/main/java/org/onap/music/mdbc/mixins/PostgresMixin.java
index 4afaa71..6d2d4cf 100755
--- a/mdbc-server/src/main/java/org/onap/music/mdbc/mixins/PostgresMixin.java
+++ b/mdbc-server/src/main/java/org/onap/music/mdbc/mixins/PostgresMixin.java
@@ -817,7 +817,7 @@ public class PostgresMixin implements DBInterface {
* @param transaction - base 64 encoded, serialized digest
*/
@Override
- public void replayTransaction(StagingTable transaction, Set<Range> ranges)
+ public void replayTransaction(StagingTable transaction)
throws SQLException, MDBCServiceException {
boolean autocommit = jdbcConn.getAutoCommit();
jdbcConn.setAutoCommit(false);
@@ -825,7 +825,6 @@ public class PostgresMixin implements DBInterface {
final ArrayList<Operation> opList = transaction.getOperationList();
for (Operation op : opList) {
- if (Range.overlaps(ranges, op.getTable())) {
try {
replayOperationIntoDB(jdbcStmt, op);
} catch (SQLException | MDBCServiceException e) {
@@ -835,7 +834,6 @@ public class PostgresMixin implements DBInterface {
jdbcConn.rollback();
throw e;
}
- }
}
clearReplayedOperations(jdbcStmt);
@@ -859,8 +857,8 @@ public class PostgresMixin implements DBInterface {
}
@Override
- public void applyTxDigest(StagingTable txDigest, Set<Range> ranges) throws SQLException, MDBCServiceException {
- replayTransaction(txDigest, ranges);
+ public void applyTxDigest(StagingTable txDigest) throws SQLException, MDBCServiceException {
+ replayTransaction(txDigest);
}
/**
diff --git a/mdbc-server/src/main/java/org/onap/music/mdbc/ownership/OwnershipAndCheckpoint.java b/mdbc-server/src/main/java/org/onap/music/mdbc/ownership/OwnershipAndCheckpoint.java
index b848964..c95644b 100644
--- a/mdbc-server/src/main/java/org/onap/music/mdbc/ownership/OwnershipAndCheckpoint.java
+++ b/mdbc-server/src/main/java/org/onap/music/mdbc/ownership/OwnershipAndCheckpoint.java
@@ -171,10 +171,10 @@ public class OwnershipAndCheckpoint{
}
}
- private void applyTxDigest(Set<Range> ranges, DBInterface di, StagingTable txDigest)
+ private void applyTxDigest(DBInterface di, StagingTable txDigest)
throws MDBCServiceException {
try {
- di.applyTxDigest(txDigest,ranges);
+ di.applyTxDigest(txDigest);
} catch (SQLException e) {
throw new MDBCServiceException("Error applying tx digest in local SQL",e);
}
@@ -213,7 +213,7 @@ public class OwnershipAndCheckpoint{
checkpointLock.unlock();
break;
} else {
- applyDigestAndUpdateDataStructures(mi, di, rangesToWarmup, node, pair);
+ applyDigestAndUpdateDataStructures(mi, di, node, pair);
}
pair = node.nextNotAppliedTransaction(rangeSet);
enableForeignKeys(di);
@@ -228,15 +228,14 @@ public class OwnershipAndCheckpoint{
}
/**
- * Apply tx digest for ranges, update checkpoint location (alreadyApplied)
+ * Apply tx digest for dagnode update checkpoint location (alreadyApplied)
* @param mi
* @param di
- * @param ranges
* @param node
* @param pair
* @throws MDBCServiceException
*/
- private void applyDigestAndUpdateDataStructures(MusicInterface mi, DBInterface di, Set<Range> ranges, DagNode node,
+ private void applyDigestAndUpdateDataStructures(MusicInterface mi, DBInterface di, DagNode node,
Pair<MusicTxDigestId, List<Range>> pair) throws MDBCServiceException {
final StagingTable txDigest;
try {
@@ -247,7 +246,7 @@ public class OwnershipAndCheckpoint{
+" case for txID "+pair.getKey().transactionId.toString());
return;
}
- applyTxDigest(ranges,di, txDigest);
+ applyTxDigest(di, txDigest);
for (Range r : pair.getValue()) {
MusicRangeInformationRow row = node.getRow();
alreadyApplied.put(r, Pair.of(new MriReference(row.getPartitionIndex()), pair.getKey().index));
@@ -287,7 +286,7 @@ public class OwnershipAndCheckpoint{
if(node!=null) {
Pair<MusicTxDigestId, List<Range>> pair = node.nextNotAppliedTransaction(rangeSet);
while (pair != null) {
- applyDigestAndUpdateDataStructures(mi, db, ranges, node, pair);
+ applyDigestAndUpdateDataStructures(mi, db, node, pair);
pair = node.nextNotAppliedTransaction(rangeSet);
if (timeout(ownOpId)) {
enableForeignKeys(db);
@@ -374,6 +373,12 @@ public class OwnershipAndCheckpoint{
false, partition.getSnapshot()));
} else if ( ownershipLocks.containsKey(uuidToOwn) || !row.getIsLatest() ) {
toOwn.setOwn(node);
+ if (ownershipLocks.containsKey(uuidToOwn) && !row.getIsLatest()) {
+ //previously owned partition that is no longer latest, don't need anymore
+ LockResult result = ownershipLocks.get(uuidToOwn);
+ ownershipLocks.remove(uuidToOwn);
+ mi.relinquish(result.getLockId(), uuidToOwn.toString());
+ }
} else {
LockRequest request = new LockRequest(uuidToOwn,
new ArrayList<>(node.getRangeSet()), lockType);
diff --git a/mdbc-server/src/main/java/org/onap/music/mdbc/tables/MusicTxDigestDaemon.java b/mdbc-server/src/main/java/org/onap/music/mdbc/tables/MusicTxDigestDaemon.java
index 6f95d3c..a1ef346 100644
--- a/mdbc-server/src/main/java/org/onap/music/mdbc/tables/MusicTxDigestDaemon.java
+++ b/mdbc-server/src/main/java/org/onap/music/mdbc/tables/MusicTxDigestDaemon.java
@@ -60,8 +60,7 @@ public class MusicTxDigestDaemon implements Runnable {
for (UUID txTimeID : keys) {
transaction = ecDigestInformation.get(txTimeID);
try {
- dbi.replayTransaction(transaction,
- ranges); // I think this Might change if the data is coming from a new table.. ( what is the new table structure??)
+ dbi.replayTransaction(transaction); // I think this Might change if the data is coming from a new table.. ( what is the new table structure??)
} catch (SQLException e) {
logger.error("EC:Rolling back the entire digest replay.");
return;
diff --git a/mdbc-server/src/main/resources/mdbc.properties b/mdbc-server/src/main/resources/mdbc.properties
index 49fdfd2..4d69da6 100755
--- a/mdbc-server/src/main/resources/mdbc.properties
+++ b/mdbc-server/src/main/resources/mdbc.properties
@@ -15,4 +15,8 @@ DEFAULT_DRIVERS=\
org.mariadb.jdbc.Driver \
org.postgresql.Driver
-txdaemonsleeps=15
+# whether or not to split the partitions
+partition_splitting=true
+
+#time, in seconds, between when the daemon catches up
+txdaemonsleeps=15 \ No newline at end of file
diff --git a/mdbc-server/src/test/java/org/onap/music/mdbc/MdbcTestUtils.java b/mdbc-server/src/test/java/org/onap/music/mdbc/MdbcTestUtils.java
index 72ec8d3..f7dd8ee 100644
--- a/mdbc-server/src/test/java/org/onap/music/mdbc/MdbcTestUtils.java
+++ b/mdbc-server/src/test/java/org/onap/music/mdbc/MdbcTestUtils.java
@@ -68,7 +68,7 @@ public class MdbcTestUtils {
final private static String nodeInfoTableName = "nodeinfo";
//Mariadb variables
static DB db=null;
- final public static String mariaDBDatabaseName="test";
+ final public static String mariaDBDatabaseName="TEST";
final static Integer mariaDbPort=13306;
diff --git a/mdbc-server/src/test/java/org/onap/music/mdbc/mixins/MySQLMixinTest.java b/mdbc-server/src/test/java/org/onap/music/mdbc/mixins/MySQLMixinTest.java
index 1f2c1dd..bd493c7 100644
--- a/mdbc-server/src/test/java/org/onap/music/mdbc/mixins/MySQLMixinTest.java
+++ b/mdbc-server/src/test/java/org/onap/music/mdbc/mixins/MySQLMixinTest.java
@@ -72,7 +72,7 @@ public class MySQLMixinTest {
@Test
public void testGetDataBaseName() throws SQLException {
- Assert.assertEquals(MdbcTestUtils.getMariaDBDBName(), mysqlMixin.getDatabaseName());
+ Assert.assertEquals(MdbcTestUtils.getMariaDBDBName().toUpperCase(), mysqlMixin.getDatabaseName());
}
}
diff --git a/mdbc-server/src/test/java/org/onap/music/mdbc/mixins/PostgresMixinTest.java b/mdbc-server/src/test/java/org/onap/music/mdbc/mixins/PostgresMixinTest.java
index a1cf2b1..d44ec8f 100644
--- a/mdbc-server/src/test/java/org/onap/music/mdbc/mixins/PostgresMixinTest.java
+++ b/mdbc-server/src/test/java/org/onap/music/mdbc/mixins/PostgresMixinTest.java
@@ -211,7 +211,7 @@ public class PostgresMixinTest {
Set<Range> ranges = new HashSet<>();
ranges.add(new Range("public.testtable"));
try {
- mixin.applyTxDigest(st,ranges);
+ mixin.applyTxDigest(st);
} catch (SQLException|MDBCServiceException e) {
e.printStackTrace();
fail();