diff options
6 files changed, 599 insertions, 548 deletions
diff --git a/datarouter-node/src/main/java/org/onap/dmaap/datarouter/node/TaskList.java b/datarouter-node/src/main/java/org/onap/dmaap/datarouter/node/TaskList.java index 7fa0dc4d..a77277f2 100644 --- a/datarouter-node/src/main/java/org/onap/dmaap/datarouter/node/TaskList.java +++ b/datarouter-node/src/main/java/org/onap/dmaap/datarouter/node/TaskList.java @@ -38,7 +38,7 @@ import java.util.Iterator; * called. * </ul> */ -public class TaskList { +class TaskList { private Iterator<Runnable> runlist; private HashSet<Runnable> tasks = new HashSet<>(); @@ -50,7 +50,7 @@ public class TaskList { /** * Start executing the sequence of tasks. */ - public synchronized void startRun() { + synchronized void startRun() { sofar = new HashSet<>(); added = new HashSet<>(); removed = new HashSet<>(); @@ -61,7 +61,7 @@ public class TaskList { /** * Get the next task to execute. */ - public synchronized Runnable next() { + synchronized Runnable next() { while (runlist != null) { if (runlist.hasNext()) { Runnable task = runlist.next(); @@ -88,7 +88,7 @@ public class TaskList { /** * Add a task to the list of tasks to run whenever the event occurs. */ - public synchronized void addTask(Runnable task) { + synchronized void addTask(Runnable task) { if (runlist != null) { added.add(task); removed.remove(task); @@ -99,7 +99,7 @@ public class TaskList { /** * Remove a task from the list of tasks to run whenever the event occurs. */ - public synchronized void removeTask(Runnable task) { + synchronized void removeTask(Runnable task) { if (runlist != null) { removed.add(task); added.remove(task); diff --git a/datarouter-node/src/test/java/org/onap/dmaap/datarouter/node/TaskListTest.java b/datarouter-node/src/test/java/org/onap/dmaap/datarouter/node/TaskListTest.java new file mode 100644 index 00000000..311165c6 --- /dev/null +++ b/datarouter-node/src/test/java/org/onap/dmaap/datarouter/node/TaskListTest.java @@ -0,0 +1,44 @@ +/*- + * ============LICENSE_START======================================================= + * Copyright (C) 2019 Nordix Foundation. + * ================================================================================ + * Licensed 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. + * + * SPDX-License-Identifier: Apache-2.0 + * ============LICENSE_END========================================================= + */ + +package org.onap.dmaap.datarouter.node; + +import org.junit.Test; + +public class TaskListTest { + + @Test + public void Given_New_Task_List_Verify_Add_And_Run() { + TaskList taskList = new TaskList(); + taskList.startRun(); + taskList.addTask(() -> { + }); + taskList.next(); + taskList.removeTask(() -> { + }); + } + + @Test + public void Given_Empty_Task_List_Verify_Next() { + TaskList taskList = new TaskList(); + taskList.startRun(); + taskList.next(); + } +} diff --git a/datarouter-prov/src/main/java/org/onap/dmaap/datarouter/provisioning/beans/Parameters.java b/datarouter-prov/src/main/java/org/onap/dmaap/datarouter/provisioning/beans/Parameters.java index fff10ac7..357444e4 100644 --- a/datarouter-prov/src/main/java/org/onap/dmaap/datarouter/provisioning/beans/Parameters.java +++ b/datarouter-prov/src/main/java/org/onap/dmaap/datarouter/provisioning/beans/Parameters.java @@ -57,6 +57,7 @@ public class Parameters extends Syncable { public static final String PROV_POKETIMER2 = "PROV_POKETIMER2";
public static final String PROV_SPECIAL_SUBNET = "PROV_SPECIAL_SUBNET";
public static final String PROV_LOG_RETENTION = "PROV_LOG_RETENTION";
+ public static final String DEFAULT_LOG_RETENTION = "DEFAULT_LOG_RETENTION";
public static final String NODES = "NODES";
public static final String ACTIVE_POD = "ACTIVE_POD";
public static final String STANDBY_POD = "STANDBY_POD";
diff --git a/datarouter-prov/src/main/java/org/onap/dmaap/datarouter/provisioning/utils/LogfileLoader.java b/datarouter-prov/src/main/java/org/onap/dmaap/datarouter/provisioning/utils/LogfileLoader.java index 3ba1a151..c78a5b10 100644 --- a/datarouter-prov/src/main/java/org/onap/dmaap/datarouter/provisioning/utils/LogfileLoader.java +++ b/datarouter-prov/src/main/java/org/onap/dmaap/datarouter/provisioning/utils/LogfileLoader.java @@ -1,501 +1,498 @@ -/*******************************************************************************
- * ============LICENSE_START==================================================
- * * org.onap.dmaap
- * * ===========================================================================
- * * Copyright © 2017 AT&T Intellectual Property. All rights reserved.
- * * ===========================================================================
- * * Licensed 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.
- * * ============LICENSE_END====================================================
- * *
- * * ECOMP is a trademark and service mark of AT&T Intellectual Property.
- * *
- ******************************************************************************/
-
-
-package org.onap.dmaap.datarouter.provisioning.utils;
-
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileNotFoundException;
-import java.io.FileReader;
-import java.io.FilenameFilter;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.io.LineNumberReader;
-import java.io.Reader;
-import java.sql.Connection;
-import java.sql.PreparedStatement;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.sql.Statement;
-import java.text.ParseException;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.TreeSet;
-import java.util.zip.GZIPInputStream;
-
-import com.att.eelf.configuration.EELFLogger;
-import com.att.eelf.configuration.EELFManager;
-import org.onap.dmaap.datarouter.provisioning.BaseServlet;
-import org.onap.dmaap.datarouter.provisioning.beans.DeliveryExtraRecord;
-import org.onap.dmaap.datarouter.provisioning.beans.DeliveryRecord;
-import org.onap.dmaap.datarouter.provisioning.beans.ExpiryRecord;
-import org.onap.dmaap.datarouter.provisioning.beans.Loadable;
-import org.onap.dmaap.datarouter.provisioning.beans.LogRecord;
-import org.onap.dmaap.datarouter.provisioning.beans.Parameters;
-import org.onap.dmaap.datarouter.provisioning.beans.PubFailRecord;
-import org.onap.dmaap.datarouter.provisioning.beans.PublishRecord;
-
-/**
- * This class provides methods that run in a separate thread, in order to process logfiles uploaded into the spooldir.
- * These logfiles are loaded into the MariaDB LOG_RECORDS table. In a running provisioning server, there should only be
- * two places where records can be loaded into this table; here, and in the method DB.retroFit4() which may be run at
- * startup to load the old (1.0) style log tables into LOG_RECORDS;
- * <p>This method maintains an {@link RLEBitSet} which can be used to easily see what records are presently in the
- * database.
- * This bit set is used to synchronize between provisioning servers.</p>
- *
- * @author Robert Eby
- * @version $Id: LogfileLoader.java,v 1.22 2014/03/12 19:45:41 eby Exp $
- */
-public class LogfileLoader extends Thread {
- /**
- * Default number of log records to keep when pruning. Keep 10M by default.
- */
- public static final long DEFAULT_LOG_RETENTION = 10000000L;
- /**
- * NOT USED: Percentage of free space required before old records are removed.
- */
- public static final int REQUIRED_FREE_PCT = 20;
-
- /**
- * This is a singleton -- there is only one LogfileLoader object in the server
- */
- private static LogfileLoader logfileLoader;
-
- /**
- * Get the singleton LogfileLoader object, and start it if it is not running.
- *
- * @return the LogfileLoader
- */
- public static synchronized LogfileLoader getLoader() {
- if (logfileLoader == null)
- logfileLoader = new LogfileLoader();
- if (!logfileLoader.isAlive())
- logfileLoader.start();
- return logfileLoader;
- }
-
- /**
- * The PreparedStatement which is loaded by a <i>Loadable</i>.
- */
- public static final String INSERT_SQL = "insert into LOG_RECORDS values (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
- /**
- * Each server can assign this many IDs
- */
- private static final long SET_SIZE = (1L << 56);
-
- private final EELFLogger logger;
- private final DB db;
- private final String spooldir;
- private final long set_start;
- private final long set_end;
- private RLEBitSet seq_set;
- private long nextid;
- private boolean idle;
-
- private LogfileLoader() {
- this.logger = EELFManager.getInstance().getLogger("InternalLog");
- this.db = new DB();
- this.spooldir = db.getProperties().getProperty("org.onap.dmaap.datarouter.provserver.spooldir");
- this.set_start = getIdRange();
- this.set_end = set_start + SET_SIZE - 1;
- this.seq_set = new RLEBitSet();
- this.nextid = 0;
- this.idle = false;
-
- // This is a potentially lengthy operation, so has been moved to run()
- //initializeNextid();
- this.setDaemon(true);
- this.setName("LogfileLoader");
- }
-
- private long getIdRange() {
- long n;
- if (BaseServlet.isInitialActivePOD())
- n = 0;
- else if (BaseServlet.isInitialStandbyPOD())
- n = SET_SIZE;
- else
- n = SET_SIZE * 2;
- String r = String.format("[%X .. %X]", n, n + SET_SIZE - 1);
- logger.debug("This server shall assign RECORD_IDs in the range " + r);
- return n;
- }
-
- /**
- * Return the bit set representing the record ID's that are loaded in this database.
- *
- * @return the bit set
- */
- public RLEBitSet getBitSet() {
- return seq_set;
- }
-
- /**
- * True if the LogfileLoader is currently waiting for work.
- *
- * @return true if idle
- */
- public boolean isIdle() {
- return idle;
- }
-
- /**
- * Run continuously to look for new logfiles in the spool directory and import them into the DB.
- * The spool is checked once per second. If free space on the MariaDB filesystem falls below
- * REQUIRED_FREE_PCT (normally 20%) then the oldest logfile entries are removed and the LOG_RECORDS
- * table is compacted until free space rises above the threshold.
- */
- @Override
- public void run() {
- initializeNextid(); // moved from the constructor
- while (true) {
- try {
- File dirfile = new File(spooldir);
- while (true) {
- // process IN files
- File[] infiles = dirfile.listFiles(new FilenameFilter() {
- @Override
- public boolean accept(File dir, String name) {
- return name.startsWith("IN.");
- }
- });
-
- if (infiles.length == 0) {
- idle = true;
- try {
- Thread.sleep(1000L);
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- }
- idle = false;
- } else {
- // Remove old rows
- if (pruneRecords()) {
- // Removed at least some entries, recompute the bit map
- initializeNextid();
- }
-
- // Process incoming logfiles
- for (File f : infiles) {
- if (logger.isDebugEnabled())
- logger.debug("PROV8001 Starting " + f + " ...");
- long time = System.currentTimeMillis();
- int[] n = process(f);
- time = System.currentTimeMillis() - time;
- logger.info(String
- .format("PROV8000 Processed %s in %d ms; %d of %d records.",
- f.toString(), time, n[0], n[1]));
- f.delete();
- }
- }
- }
- } catch (Exception e) {
- logger.warn("PROV0020: Caught exception in LogfileLoader: " + e);
- }
- }
- }
-
- boolean pruneRecords() {
- boolean did1 = false;
- long count = countRecords();
- long threshold = DEFAULT_LOG_RETENTION;
- Parameters param = Parameters.getParameter(Parameters.PROV_LOG_RETENTION);
- if (param != null) {
- try {
- long n = Long.parseLong(param.getValue());
- // This check is to prevent inadvertent errors from wiping the table out
- if (n > 1000000L)
- threshold = n;
- } catch (NumberFormatException e) {
- // ignore
- }
- }
- logger.debug("Pruning LOG_RECORD table: records in DB=" + count + ", threshold=" + threshold);
- if (count > threshold) {
- count -= threshold; // we need to remove this many records;
- Map<Long, Long> hist = getHistogram(); // histogram of records per day
- // Determine the cutoff point to remove the needed number of records
- long sum = 0;
- long cutoff = 0;
- for (Long day : new TreeSet<Long>(hist.keySet())) {
- sum += hist.get(day);
- cutoff = day;
- if (sum >= count)
- break;
- }
- cutoff++;
- cutoff *= 86400000L; // convert day to ms
- logger.debug(" Pruning records older than=" + (cutoff / 86400000L) + " (" + new Date(cutoff) + ")");
-
- Connection conn = null;
- try {
- // Limit to a million at a time to avoid typing up the DB for too long.
- conn = db.getConnection();
- try(PreparedStatement ps = conn.prepareStatement("DELETE from LOG_RECORDS where EVENT_TIME < ? limit 1000000")) {
- ps.setLong(1, cutoff);
- while (count > 0) {
- if (!ps.execute()) {
- int dcount = ps.getUpdateCount();
- count -= dcount;
- logger.debug(" " + dcount + " rows deleted.");
- did1 |= (dcount != 0);
- if (dcount == 0)
- count = 0; // prevent inf. loops
- } else {
- count = 0; // shouldn't happen!
- }
- }
- }
- try(Statement stmt = conn.createStatement()) {
- stmt.execute("OPTIMIZE TABLE LOG_RECORDS");
- }
- } catch (SQLException e) {
- System.err.println(e);
- logger.error(e.toString());
- } finally {
- db.release(conn);
- }
- }
- return did1;
- }
-
- long countRecords() {
- long count = 0;
- Connection conn = null;
- try {
- conn = db.getConnection();
- try(Statement stmt = conn.createStatement()) {
- try(ResultSet rs = stmt.executeQuery("SELECT COUNT(*) as COUNT from LOG_RECORDS")) {
- if (rs.next()) {
- count = rs.getLong("COUNT");
- }
- }
- }
- } catch (SQLException e) {
- System.err.println(e);
- logger.error(e.toString());
- } finally {
- db.release(conn);
- }
- return count;
- }
-
- Map<Long, Long> getHistogram() {
- Map<Long, Long> map = new HashMap<Long, Long>();
- Connection conn = null;
- try {
- logger.debug(" LOG_RECORD table histogram...");
- conn = db.getConnection();
- try(Statement stmt = conn.createStatement()) {
- try(ResultSet rs = stmt.executeQuery("SELECT FLOOR(EVENT_TIME/86400000) AS DAY, COUNT(*) AS COUNT FROM LOG_RECORDS GROUP BY DAY")) {
- while (rs.next()) {
- long day = rs.getLong("DAY");
- long cnt = rs.getLong("COUNT");
- map.put(day, cnt);
- logger.debug(" " + day + " " + cnt);
- }
- }
- }
- } catch (SQLException e) {
- System.err.println(e);
- logger.error(e.toString());
- } finally {
- db.release(conn);
- }
- return map;
- }
-
- private void initializeNextid() {
- Connection conn = null;
- try {
- conn = db.getConnection();
- RLEBitSet nbs = new RLEBitSet();
- try(Statement stmt = conn.createStatement()) {
- // Build a bitset of all records in the LOG_RECORDS table
- // We need to run this SELECT in stages, because otherwise we run out of memory!
- final long stepsize = 6000000L;
- boolean go_again = true;
- for (long i = 0; go_again; i += stepsize) {
- String sql = String.format("select RECORD_ID from LOG_RECORDS LIMIT %d,%d", i, stepsize);
- try (ResultSet rs = stmt.executeQuery(sql)) {
- go_again = false;
- while (rs.next()) {
- long n = rs.getLong("RECORD_ID");
- nbs.set(n);
- go_again = true;
- }
- }
- }
- }
- seq_set = nbs;
- // Compare with the range for this server
- // Determine the next ID for this set of record IDs
- RLEBitSet tbs = (RLEBitSet) nbs.clone();
- RLEBitSet idset = new RLEBitSet();
- idset.set(set_start, set_start + SET_SIZE);
- tbs.and(idset);
- long t = tbs.length();
- nextid = (t == 0) ? set_start : (t - 1);
- if (nextid >= set_start + SET_SIZE) {
- // Handle wraparound, when the IDs reach the end of our "range"
- Long[] last = null;
- Iterator<Long[]> li = tbs.getRangeIterator();
- while (li.hasNext()) {
- last = li.next();
- }
- if (last != null) {
- tbs.clear(last[0], last[1] + 1);
- t = tbs.length();
- nextid = (t == 0) ? set_start : (t - 1);
- }
- }
- logger.debug(String.format("initializeNextid, next ID is %d (%x)", nextid, nextid));
- } catch (SQLException e) {
- System.err.println(e);
- logger.error(e.toString());
- } finally {
- db.release(conn);
- }
- }
-
- @SuppressWarnings("resource")
- int[] process(File f) {
- int ok = 0, total = 0;
- try {
- Connection conn = db.getConnection();
- PreparedStatement ps = conn.prepareStatement(INSERT_SQL);
- Reader r = f.getPath().endsWith(".gz")
- ? new InputStreamReader(new GZIPInputStream(new FileInputStream(f)))
- : new FileReader(f);
- try(LineNumberReader in = new LineNumberReader(r)) {
- String line;
- while ((line = in.readLine()) != null) {
- try {
- for (Loadable rec : buildRecords(line)) {
- rec.load(ps);
- if (rec instanceof LogRecord) {
- LogRecord lr = ((LogRecord) rec);
- if (!seq_set.get(lr.getRecordId())) {
- ps.executeUpdate();
- seq_set.set(lr.getRecordId());
- } else
- logger.debug("Duplicate record ignored: " + lr.getRecordId());
- } else {
- if (++nextid > set_end)
- nextid = set_start;
- ps.setLong(18, nextid);
- ps.executeUpdate();
- seq_set.set(nextid);
- }
- ps.clearParameters();
- ok++;
- }
- } catch (SQLException e) {
- logger.warn("PROV8003 Invalid value in record: " + line);
- logger.debug(e.toString(), e);
- } catch (NumberFormatException e) {
- logger.warn("PROV8004 Invalid number in record: " + line);
- logger.debug(e.toString());
- } catch (ParseException e) {
- logger.warn("PROV8005 Invalid date in record: " + line);
- logger.debug(e.toString());
- } catch (Exception e) {
- logger.warn("PROV8006 Invalid pattern in record: " + line);
- logger.debug(e.toString(), e);
- }
- total++;
- }
- }
- ps.close();
- db.release(conn);
- conn = null;
- } catch (FileNotFoundException e) {
- logger.warn("PROV8007 Exception reading " + f + ": " + e);
- } catch (IOException e) {
- logger.warn("PROV8007 Exception reading " + f + ": " + e);
- } catch (SQLException e) {
- logger.warn("PROV8007 Exception reading " + f + ": " + e);
- }
- return new int[]{ok, total};
- }
-
- Loadable[] buildRecords(String line) throws ParseException {
- String[] pp = line.split("\\|");
- if (pp != null && pp.length >= 7) {
- String rtype = pp[1].toUpperCase();
- if (rtype.equals("PUB") && pp.length == 11) {
- // Fields are: date|PUB|pubid|feedid|requrl|method|ctype|clen|srcip|user|status
- return new Loadable[]{new PublishRecord(pp)};
- }
- if (rtype.equals("DEL") && pp.length == 12) {
- // Fields are: date|DEL|pubid|feedid|subid|requrl|method|ctype|clen|user|status|xpubid
- String[] subs = pp[4].split("\\s+");
- if (subs != null) {
- Loadable[] rv = new Loadable[subs.length];
- for (int i = 0; i < subs.length; i++) {
- // create a new record for each individual sub
- pp[4] = subs[i];
- rv[i] = new DeliveryRecord(pp);
- }
- return rv;
- }
- }
- if (rtype.equals("EXP") && pp.length == 11) {
- // Fields are: date|EXP|pubid|feedid|subid|requrl|method|ctype|clen|reason|attempts
- ExpiryRecord e = new ExpiryRecord(pp);
- if (e.getReason().equals("other"))
- logger.info("Invalid reason '" + pp[9] + "' changed to 'other' for record: " + e.getPublishId());
- return new Loadable[]{e};
- }
- if (rtype.equals("PBF") && pp.length == 12) {
- // Fields are: date|PBF|pubid|feedid|requrl|method|ctype|clen-expected|clen-received|srcip|user|error
- return new Loadable[]{new PubFailRecord(pp)};
- }
- if (rtype.equals("DLX") && pp.length == 7) {
- // Fields are: date|DLX|pubid|feedid|subid|clen-tosend|clen-sent
- return new Loadable[]{new DeliveryExtraRecord(pp)};
- }
- if (rtype.equals("LOG") && (pp.length == 19 || pp.length == 20)) {
- // Fields are: date|LOG|pubid|feedid|requrl|method|ctype|clen|type|feedFileid|remoteAddr|user|status|subid|fileid|result|attempts|reason|record_id
- return new Loadable[]{new LogRecord(pp)};
- }
- }
- logger.warn("PROV8002 bad record: " + line);
- return new Loadable[0];
- }
-
- /**
- * The LogfileLoader can be run stand-alone by invoking the main() method of this class.
- *
- * @param a ignored
- * @throws InterruptedException
- */
- public static void main(String[] a) throws InterruptedException {
- LogfileLoader.getLoader();
- Thread.sleep(200000L);
- }
-}
+/******************************************************************************* + * ============LICENSE_START================================================== + * * org.onap.dmaap + * * =========================================================================== + * * Copyright © 2017 AT&T Intellectual Property. All rights reserved. + * * =========================================================================== + * * Licensed 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. + * * ============LICENSE_END==================================================== + * * + * * ECOMP is a trademark and service mark of AT&T Intellectual Property. + * * + ******************************************************************************/ + + +package org.onap.dmaap.datarouter.provisioning.utils; + +import com.att.eelf.configuration.EELFLogger; +import com.att.eelf.configuration.EELFManager; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileReader; +import java.io.IOException; +import java.io.InputStreamReader; +import java.io.LineNumberReader; +import java.io.Reader; +import java.nio.file.Files; +import java.sql.Connection; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.text.ParseException; +import java.util.Date; +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; +import java.util.TreeSet; +import java.util.zip.GZIPInputStream; +import org.onap.dmaap.datarouter.provisioning.BaseServlet; +import org.onap.dmaap.datarouter.provisioning.beans.DeliveryExtraRecord; +import org.onap.dmaap.datarouter.provisioning.beans.DeliveryRecord; +import org.onap.dmaap.datarouter.provisioning.beans.ExpiryRecord; +import org.onap.dmaap.datarouter.provisioning.beans.Loadable; +import org.onap.dmaap.datarouter.provisioning.beans.LogRecord; +import org.onap.dmaap.datarouter.provisioning.beans.Parameters; +import org.onap.dmaap.datarouter.provisioning.beans.PubFailRecord; +import org.onap.dmaap.datarouter.provisioning.beans.PublishRecord; + +/** + * This class provides methods that run in a separate thread, in order to process logfiles uploaded into the spooldir. + * These logfiles are loaded into the MariaDB LOG_RECORDS table. In a running provisioning server, there should only be + * two places where records can be loaded into this table; here, and in the method DB.retroFit4() which may be run at + * startup to load the old (1.0) style log tables into LOG_RECORDS; + * <p>This method maintains an {@link RLEBitSet} which can be used to easily see what records are presently in the + * database. + * This bit set is used to synchronize between provisioning servers.</p> + * + * @author Robert Eby + * @version $Id: LogfileLoader.java,v 1.22 2014/03/12 19:45:41 eby Exp $ + */ +public class LogfileLoader extends Thread { + /** + * NOT USED: Percentage of free space required before old records are removed. + */ + public static final int REQUIRED_FREE_PCT = 20; + + /** + * This is a singleton -- there is only one LogfileLoader object in the server. + */ + private static LogfileLoader logfileLoader; + + /** + * The PreparedStatement which is loaded by a <i>Loadable</i>. + */ + private static final String INSERT_SQL = "insert into LOG_RECORDS values (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)"; + /** + * Each server can assign this many IDs. + */ + private static final long SET_SIZE = (1L << 56); + + private final EELFLogger logger; + private final DB db; + private final String spooldir; + private final long setStart; + private final long setEnd; + private RLEBitSet seqSet; + private long nextId; + private boolean idle; + + /** + * Get the singleton LogfileLoader object, and start it if it is not running. + * + * @return the LogfileLoader + */ + public static synchronized LogfileLoader getLoader() { + if (logfileLoader == null) { + logfileLoader = new LogfileLoader(); + } + if (!logfileLoader.isAlive()) { + logfileLoader.start(); + } + return logfileLoader; + } + + + private LogfileLoader() { + this.logger = EELFManager.getInstance().getLogger("InternalLog"); + this.db = new DB(); + this.spooldir = db.getProperties().getProperty("org.onap.dmaap.datarouter.provserver.spooldir"); + this.setStart = getIdRange(); + this.setEnd = setStart + SET_SIZE - 1; + this.seqSet = new RLEBitSet(); + this.nextId = 0; + this.idle = false; + this.setDaemon(true); + this.setName("LogfileLoader"); + } + + private long getIdRange() { + long n; + if (BaseServlet.isInitialActivePOD()) { + n = 0; + } else if (BaseServlet.isInitialStandbyPOD()) { + n = SET_SIZE; + } else { + n = SET_SIZE * 2; + } + String r = String.format("[%X .. %X]", n, n + SET_SIZE - 1); + logger.debug("This server shall assign RECORD_IDs in the range " + r); + return n; + } + + /** + * Return the bit set representing the record ID's that are loaded in this database. + * + * @return the bit set + */ + public RLEBitSet getBitSet() { + return seqSet; + } + + /** + * True if the LogfileLoader is currently waiting for work. + * + * @return true if idle + */ + public boolean isIdle() { + return idle; + } + + /** + * Run continuously to look for new logfiles in the spool directory and import them into the DB. + * The spool is checked once per second. If free space on the MariaDB filesystem falls below + * REQUIRED_FREE_PCT (normally 20%) then the oldest logfile entries are removed and the LOG_RECORDS + * table is compacted until free space rises above the threshold. + */ + @Override + public void run() { + initializeNextid(); + while (true) { + try { + File dirfile = new File(spooldir); + while (true) { + runLogFileLoad(dirfile); + } + } catch (Exception e) { + logger.warn("PROV0020: Caught exception in LogfileLoader: " + e); + } + } + } + + private void runLogFileLoad(File filesDir) { + File[] inFiles = filesDir.listFiles((dir, name) -> name.startsWith("IN.")); + if (inFiles != null) { + if (inFiles.length == 0) { + idle = true; + try { + Thread.sleep(1000L); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + idle = false; + } else { + // Remove old rows + if (pruneRecords()) { + // Removed at least some entries, recompute the bit map + initializeNextid(); + } + for (File file : inFiles) { + processFile(file); + } + } + } + } + + private void processFile(File infile) { + if (logger.isDebugEnabled()) { + logger.debug("PROV8001 Starting " + infile + " ..."); + } + long time = System.currentTimeMillis(); + int[] n = process(infile); + time = System.currentTimeMillis() - time; + logger.info(String.format("PROV8000 Processed %s in %d ms; %d of %d records.", + infile.toString(), time, n[0], n[1])); + try { + Files.delete(infile.toPath()); + } catch (IOException e) { + logger.info("PROV8001 failed to delete file " + infile.getName(), e); + } + } + + boolean pruneRecords() { + boolean did1 = false; + long count = countRecords(); + Parameters defaultLogRetention = Parameters.getParameter(Parameters.DEFAULT_LOG_RETENTION); + long threshold = (defaultLogRetention != null) ? Long.parseLong(defaultLogRetention.getValue()) : 1000000L; + Parameters provLogRetention = Parameters.getParameter(Parameters.PROV_LOG_RETENTION); + if (provLogRetention != null) { + try { + long n = Long.parseLong(provLogRetention.getValue()); + // This check is to prevent inadvertent errors from wiping the table out + if (n > 1000000L) { + threshold = n; + } + } catch (NumberFormatException e) { + // ignore + } + } + logger.debug("Pruning LOG_RECORD table: records in DB=" + count + ", threshold=" + threshold); + if (count > threshold) { + // we need to remove this many records + count -= threshold; + // histogram of records per day + Map<Long, Long> hist = getHistogram(); + // Determine the cutoff point to remove the needed number of records + long sum = 0; + long cutoff = 0; + for (Long day : new TreeSet<>(hist.keySet())) { + sum += hist.get(day); + cutoff = day; + if (sum >= count) { + break; + } + } + cutoff++; + // convert day to ms + cutoff *= 86400000L; + logger.debug(" Pruning records older than=" + (cutoff / 86400000L) + " (" + new Date(cutoff) + ")"); + + Connection conn = null; + try { + // Limit to a million at a time to avoid typing up the DB for too long. + conn = db.getConnection(); + try (PreparedStatement ps = conn.prepareStatement("DELETE from LOG_RECORDS where EVENT_TIME < ? limit 1000000")) { + ps.setLong(1, cutoff); + while (count > 0) { + if (!ps.execute()) { + int dcount = ps.getUpdateCount(); + count -= dcount; + logger.debug(" " + dcount + " rows deleted."); + did1 |= (dcount != 0); + if (dcount == 0) { + count = 0; // prevent inf. loops + } + } else { + count = 0; // shouldn't happen! + } + } + } + try (Statement stmt = conn.createStatement()) { + stmt.execute("OPTIMIZE TABLE LOG_RECORDS"); + } + } catch (SQLException e) { + logger.error(e.toString()); + } finally { + db.release(conn); + } + } + return did1; + } + + long countRecords() { + long count = 0; + Connection conn = null; + try { + conn = db.getConnection(); + try (Statement stmt = conn.createStatement()) { + try (ResultSet rs = stmt.executeQuery("SELECT COUNT(*) as COUNT from LOG_RECORDS")) { + if (rs.next()) { + count = rs.getLong("COUNT"); + } + } + } + } catch (SQLException e) { + logger.error(e.toString()); + } finally { + db.release(conn); + } + return count; + } + + Map<Long, Long> getHistogram() { + Map<Long, Long> map = new HashMap<>(); + Connection conn = null; + try { + logger.debug(" LOG_RECORD table histogram..."); + conn = db.getConnection(); + try (Statement stmt = conn.createStatement()) { + try (ResultSet rs = stmt.executeQuery("SELECT FLOOR(EVENT_TIME/86400000) AS DAY, COUNT(*) AS COUNT FROM LOG_RECORDS GROUP BY DAY")) { + while (rs.next()) { + long day = rs.getLong("DAY"); + long cnt = rs.getLong("COUNT"); + map.put(day, cnt); + logger.debug(" " + day + " " + cnt); + } + } + } + } catch (SQLException e) { + logger.error(e.toString()); + } finally { + db.release(conn); + } + return map; + } + + private void initializeNextid() { + Connection conn = null; + try { + conn = db.getConnection(); + RLEBitSet nbs = new RLEBitSet(); + try (Statement stmt = conn.createStatement()) { + // Build a bitset of all records in the LOG_RECORDS table + // We need to run this SELECT in stages, because otherwise we run out of memory! + final long stepsize = 6000000L; + boolean goAgain = true; + for (long i = 0; goAgain; i += stepsize) { + String sql = String.format("select RECORD_ID from LOG_RECORDS LIMIT %d,%d", i, stepsize); + try (ResultSet rs = stmt.executeQuery(sql)) { + goAgain = false; + while (rs.next()) { + long n = rs.getLong("RECORD_ID"); + nbs.set(n); + goAgain = true; + } + } + } + } + seqSet = nbs; + // Compare with the range for this server + // Determine the next ID for this set of record IDs + RLEBitSet tbs = (RLEBitSet) nbs.clone(); + RLEBitSet idset = new RLEBitSet(); + idset.set(setStart, setStart + SET_SIZE); + tbs.and(idset); + long t = tbs.length(); + nextId = (t == 0) ? setStart : (t - 1); + if (nextId >= setStart + SET_SIZE) { + // Handle wraparound, when the IDs reach the end of our "range" + Long[] last = null; + Iterator<Long[]> li = tbs.getRangeIterator(); + while (li.hasNext()) { + last = li.next(); + } + if (last != null) { + tbs.clear(last[0], last[1] + 1); + t = tbs.length(); + nextId = (t == 0) ? setStart : (t - 1); + } + } + logger.debug(String.format("initializeNextid, next ID is %d (%x)", nextId, nextId)); + } catch (SQLException e) { + logger.error(e.toString()); + } finally { + db.release(conn); + } + } + + @SuppressWarnings("resource") + int[] process(File f) { + int ok = 0; + int total = 0; + try { + Connection conn = db.getConnection(); + PreparedStatement ps = conn.prepareStatement(INSERT_SQL); + Reader r = f.getPath().endsWith(".gz") + ? new InputStreamReader(new GZIPInputStream(new FileInputStream(f))) + : new FileReader(f); + try (LineNumberReader in = new LineNumberReader(r)) { + String line; + while ((line = in.readLine()) != null) { + try { + for (Loadable rec : buildRecords(line)) { + rec.load(ps); + if (rec instanceof LogRecord) { + LogRecord lr = ((LogRecord) rec); + if (!seqSet.get(lr.getRecordId())) { + ps.executeUpdate(); + seqSet.set(lr.getRecordId()); + } else { + logger.debug("Duplicate record ignored: " + lr.getRecordId()); + } + } else { + if (++nextId > setEnd) { + nextId = setStart; + } + ps.setLong(18, nextId); + ps.executeUpdate(); + seqSet.set(nextId); + } + ps.clearParameters(); + ok++; + } + } catch (SQLException e) { + logger.warn("PROV8003 Invalid value in record: " + line, e); + } catch (NumberFormatException e) { + logger.warn("PROV8004 Invalid number in record: " + line, e); + } catch (ParseException e) { + logger.warn("PROV8005 Invalid date in record: " + line, e); + } catch (Exception e) { + logger.warn("PROV8006 Invalid pattern in record: " + line, e); + } + total++; + } + } + ps.close(); + db.release(conn); + } catch (SQLException | IOException e) { + logger.warn("PROV8007 Exception reading " + f + ": " + e); + } + return new int[]{ok, total}; + } + + Loadable[] buildRecords(String line) throws ParseException { + String[] pp = line.split("\\|"); + if (pp != null && pp.length >= 7) { + String rtype = pp[1].toUpperCase(); + if ("PUB".equals(rtype) && pp.length == 11) { + // Fields are: date|PUB|pubid|feedid|requrl|method|ctype|clen|srcip|user|status + return new Loadable[]{new PublishRecord(pp)}; + } + if ("DEL".equals(rtype) && pp.length == 12) { + // Fields are: date|DEL|pubid|feedid|subid|requrl|method|ctype|clen|user|status|xpubid + String[] subs = pp[4].split("\\s+"); + if (subs != null) { + Loadable[] rv = new Loadable[subs.length]; + for (int i = 0; i < subs.length; i++) { + // create a new record for each individual sub + pp[4] = subs[i]; + rv[i] = new DeliveryRecord(pp); + } + return rv; + } + } + if ("EXP".equals(rtype) && pp.length == 11) { + // Fields are: date|EXP|pubid|feedid|subid|requrl|method|ctype|clen|reason|attempts + ExpiryRecord e = new ExpiryRecord(pp); + if ("other".equals(e.getReason())) { + logger.info("Invalid reason '" + pp[9] + "' changed to 'other' for record: " + e.getPublishId()); + } + return new Loadable[]{e}; + } + if ("PBF".equals(rtype) && pp.length == 12) { + // Fields are: date|PBF|pubid|feedid|requrl|method|ctype|clen-expected|clen-received|srcip|user|error + return new Loadable[]{new PubFailRecord(pp)}; + } + if ("DLX".equals(rtype) && pp.length == 7) { + // Fields are: date|DLX|pubid|feedid|subid|clen-tosend|clen-sent + return new Loadable[]{new DeliveryExtraRecord(pp)}; + } + if ("LOG".equals(rtype) && (pp.length == 19 || pp.length == 20)) { + // Fields are: date|LOG|pubid|feedid|requrl|method|ctype|clen|type|feedFileid|remoteAddr|user|status|subid|fileid|result|attempts|reason|record_id + return new Loadable[]{new LogRecord(pp)}; + } + } + logger.warn("PROV8002 bad record: " + line); + return new Loadable[0]; + } + + /** + * The LogfileLoader can be run stand-alone by invoking the main() method of this class. + * + * @param a ignored + */ + public static void main(String[] a) throws InterruptedException { + LogfileLoader.getLoader(); + Thread.sleep(200000L); + } +} diff --git a/datarouter-prov/src/test/java/org/onap/dmaap/datarouter/provisioning/utils/LogfileLoaderTest.java b/datarouter-prov/src/test/java/org/onap/dmaap/datarouter/provisioning/utils/LogfileLoaderTest.java index 7f6d7de6..a15509b4 100644 --- a/datarouter-prov/src/test/java/org/onap/dmaap/datarouter/provisioning/utils/LogfileLoaderTest.java +++ b/datarouter-prov/src/test/java/org/onap/dmaap/datarouter/provisioning/utils/LogfileLoaderTest.java @@ -20,32 +20,29 @@ package org.onap.dmaap.datarouter.provisioning.utils; +import static junit.framework.TestCase.assertTrue; +import static org.junit.Assert.assertFalse; + +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.nio.file.Files; +import javax.persistence.EntityManager; +import javax.persistence.EntityManagerFactory; +import javax.persistence.Persistence; +import org.junit.After; import org.junit.AfterClass; import org.junit.Assert; +import org.junit.Before; import org.junit.BeforeClass; -import org.junit.Rule; -import org.junit.rules.TemporaryFolder; +import org.junit.Test; import org.junit.runner.RunWith; - import org.onap.dmaap.datarouter.provisioning.InternalServlet; import org.onap.dmaap.datarouter.provisioning.beans.Parameters; import org.powermock.api.mockito.PowerMockito; import org.powermock.core.classloader.annotations.SuppressStaticInitializationFor; import org.powermock.modules.junit4.PowerMockRunner; -import javax.persistence.EntityManager; -import javax.persistence.EntityManagerFactory; -import javax.persistence.Persistence; -import java.io.File; -import java.io.FileWriter; -import java.io.IOException; - -import static org.junit.Assert.assertFalse; - -import org.junit.Test; - - - @RunWith(PowerMockRunner.class) @SuppressStaticInitializationFor("org.onap.dmaap.datarouter.provisioning.beans.Parameters") public class LogfileLoaderTest { @@ -53,10 +50,18 @@ public class LogfileLoaderTest { private static EntityManagerFactory emf; private static EntityManager em; private LogfileLoader lfl = LogfileLoader.getLoader(); + private File testLog; - @Rule - public TemporaryFolder folder = new TemporaryFolder(); + @Before + public void setUp() throws Exception { + testLog = new File(System.getProperty("user.dir") + "/src/test/resources/IN.test_prov_logs"); + prepFile(testLog); + } + @After + public void tearDown() throws IOException { + Files.deleteIfExists(testLog.toPath()); + } @BeforeClass public static void init() { @@ -68,7 +73,6 @@ public class LogfileLoaderTest { InternalServlet internalServlet = new InternalServlet(); } - @AfterClass public static void tearDownClass() { em.clear(); @@ -76,42 +80,47 @@ public class LogfileLoaderTest { emf.close(); } - @Test - public void Verify_File_Processing_when_Req_Type_LOG() throws IOException { - String fileContent = "2018-08-29-10-10-10-543.|LOG|1|1|url/file123|method|1|1|type|1|128.0.0.9|user123|2|1|1|1|other|1"; - int[] actual = lfl.process(prepFile(fileContent, "file1")); - int[] expect = {0, 1}; + public void Verify_File_Processing_Returns_Expected_Array() { + int[] actual = lfl.process(testLog); + int[] expect = {5, 7}; Assert.assertArrayEquals(expect, actual); + Assert.assertNotNull(lfl.getBitSet()); + Assert.assertTrue(lfl.isIdle()); } - @Test - public void Verify_File_Processing_when_Req_Type_EXP() throws IOException{ - String fileContent = "2018-08-29-10-10-10-543.|EXP|1|1|1|'url/file123'|method|ctype|3|other|4"; - int[] actual = lfl.process(prepFile(fileContent, "file2")); - int[] expect = {0, 1}; - Assert.assertArrayEquals(expect, actual); + public void Verify_Records_Prune_When_Record_Count_Is_Less_Then_Threshold() { + lfl.process(testLog); + PowerMockito.mockStatic(Parameters.class); + PowerMockito.when(Parameters.getParameter(Parameters.PROV_LOG_RETENTION)).thenReturn(new Parameters(Parameters.PROV_LOG_RETENTION, "0")); + PowerMockito.when(Parameters.getParameter(Parameters.DEFAULT_LOG_RETENTION)).thenReturn(new Parameters(Parameters.DEFAULT_LOG_RETENTION, "1000000")); + assertFalse(lfl.pruneRecords()); } - @Test - public void Verify_Records_Prune_When_Record_Count_Is_Less_Then_Threshold() throws IOException{ - String fileContent = "2018-08-29-10-10-10-543.|PUB|1|1|https://dmaap-dr-prov:8443/publish/1/file123/|POST|application/vnd.att-dr.feed|2|128.0.0.9|user123|200"; - lfl.process(prepFile(fileContent, "file3")); + public void Verify_Records_Prune_When_Record_Count_Is_Greater_Then_Threshold() { + lfl.process(testLog); PowerMockito.mockStatic(Parameters.class); PowerMockito.when(Parameters.getParameter(Parameters.PROV_LOG_RETENTION)).thenReturn(new Parameters(Parameters.PROV_LOG_RETENTION, "0")); - assertFalse(lfl.pruneRecords()); + PowerMockito.when(Parameters.getParameter(Parameters.DEFAULT_LOG_RETENTION)).thenReturn(new Parameters(Parameters.DEFAULT_LOG_RETENTION, "1")); + assertTrue(lfl.pruneRecords()); } - private File prepFile(String content, String fileName) throws IOException{ - File file1 = folder.newFile(fileName); - try (FileWriter fileWriter = new FileWriter(file1)) { - fileWriter.write(content); - }catch (IOException e){ + private void prepFile(File logFile) { + String testLogs = "2018-08-29-10-10-10-543.|LOG|1|1|https://dmaap-dr-prov:/url/file123|POST|application/vnd.att-dr.feed|100|mockType|file123|https://dmaap-dr-prov|user123|200|1|1|200|2|2\n" + + "2018-08-29-10-10-10-543.|EXP|1|1|1|'url/file123'|PUT|null|3|new reason|4\n" + + "2018-08-29-10-10-10-543.|PUB|1|1|https://dmaap-dr-prov:8443/publish/1/file123/|POST|application/vnd.att-dr.feed|2|128.0.0.9|user123|200\n" + + "2018-08-29-10-10-10-543.|PBF|1|1|https://dmaap-dr-prov:8443/publish/1/file123/|POST|application/vnd.att-dr.feed|100|100|128.0.0.9|user123|failed\n" + + "2018-08-29-10-10-10-543.|DLX|1|1|1|100|100\n" + + "2018-08-29-10-10-10-543.|Bad Record|||\n" + + "2018-08-29-10-10-10-543.|DEL|2|1|2|https://dmaap-dr-prov:8443/publish/1/file123/|PUT|application/vnd.att-dr.feed|100|user123|200|123456"; + try (FileWriter fileWriter = new FileWriter(logFile)) { + fileWriter.write(testLogs); + } + catch (IOException e){ System.out.println(e.getMessage()); } - return file1; } } diff --git a/datarouter-prov/src/test/resources/h2Database.properties b/datarouter-prov/src/test/resources/h2Database.properties index 9c63aea4..cb472419 100755 --- a/datarouter-prov/src/test/resources/h2Database.properties +++ b/datarouter-prov/src/test/resources/h2Database.properties @@ -27,7 +27,7 @@ org.onap.dmaap.datarouter.db.url = jdbc:h2:mem:test;DB org.onap.dmaap.datarouter.provserver.isaddressauthenabled = true org.onap.dmaap.datarouter.provserver.https.relaxation = false org.onap.dmaap.datarouter.provserver.accesslog.dir = unit-test-logs -org.onap.dmaap.datarouter.provserver.spooldir = unit-test-logs/spool +org.onap.dmaap.datarouter.provserver.spooldir = src/test/resources org.onap.dmaap.datarouter.provserver.localhost = 127.0.0.1 org.onap.dmaap.datarouter.provserver.passwordencryption = PasswordEncryptionKey#@$%^&1234# |