From 75a28432cdf674c614aba2feb5942671732421ac Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Tue, 5 Dec 2023 12:12:46 +0200 Subject: [PATCH 001/146] Initial setup for Kafka consumer groups. --- pom.xml | 67 +++++++ src/main/java/com/teragrep/cfe_39/Config.java | 161 +++++++++++++++++ .../consumers/kafka/DatabaseOutput.java | 117 +++++++++++++ .../consumers/kafka/KafkaController.java | 159 +++++++++++++++++ .../consumers/kafka/KafkaReaderTemp.java | 63 +++++++ .../kafka/MockKafkaConsumerFactoryTemp.java | 163 ++++++++++++++++++ .../consumers/kafka/ReadCoordinatorTemp.java | 71 ++++++++ .../cfe_39/metrics/DurationStatistics.java | 81 +++++++++ .../cfe_39/metrics/RuntimeStatistics.java | 48 ++++++ .../cfe_39/metrics/mxj/AtomicLongMXJItem.java | 43 +++++ .../cfe_39/metrics/mxj/MXJBeanDynamizer.java | 29 ++++ .../cfe_39/metrics/mxj/MXJEndpoint.java | 40 +++++ .../teragrep/cfe_39/metrics/mxj/MXJItem.java | 14 ++ .../cfe_39/metrics/topic/TopicCounter.java | 69 ++++++++ .../cfe_39/metrics/topic/TopicStatistics.java | 59 +++++++ 15 files changed, 1184 insertions(+) create mode 100644 pom.xml create mode 100644 src/main/java/com/teragrep/cfe_39/Config.java create mode 100644 src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java create mode 100644 src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java create mode 100644 src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReaderTemp.java create mode 100644 src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java create mode 100644 src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinatorTemp.java create mode 100644 src/main/java/com/teragrep/cfe_39/metrics/DurationStatistics.java create mode 100644 src/main/java/com/teragrep/cfe_39/metrics/RuntimeStatistics.java create mode 100644 src/main/java/com/teragrep/cfe_39/metrics/mxj/AtomicLongMXJItem.java create mode 100644 src/main/java/com/teragrep/cfe_39/metrics/mxj/MXJBeanDynamizer.java create mode 100644 src/main/java/com/teragrep/cfe_39/metrics/mxj/MXJEndpoint.java create mode 100644 src/main/java/com/teragrep/cfe_39/metrics/mxj/MXJItem.java create mode 100644 src/main/java/com/teragrep/cfe_39/metrics/topic/TopicCounter.java create mode 100644 src/main/java/com/teragrep/cfe_39/metrics/topic/TopicStatistics.java diff --git a/pom.xml b/pom.xml new file mode 100644 index 00000000..6f9c7eec --- /dev/null +++ b/pom.xml @@ -0,0 +1,67 @@ + + + 4.0.0 + com.teragrep + cfe_39 + jar + ${revision}${sha1}${changelist} + cfe_39 + + UTF-8 + 1.8 + 1.8 + 1.8 + 0.0.1 + -SNAPSHOT + + + + + + + + + com.teragrep + rlo_06 + 9.0.1 + + + com.teragrep + rlo_09 + 2.0.4 + + + org.apache.avro + avro + 1.11.3 + + + org.apache.logging.log4j + log4j-slf4j2-impl + 2.20.0 + + + org.apache.logging.log4j + log4j-core + 2.19.0 + + + org.apache.logging.log4j + log4j-slf4j2-impl + 2.20.0 + + + org.slf4j + slf4j-api + 2.0.7 + + + + + scrapyard.xnet.fi-teragrep-releases + https://scrapyard.xnet.fi/repository/teragrep-releases/ + + + \ No newline at end of file diff --git a/src/main/java/com/teragrep/cfe_39/Config.java b/src/main/java/com/teragrep/cfe_39/Config.java new file mode 100644 index 00000000..17816d11 --- /dev/null +++ b/src/main/java/com/teragrep/cfe_39/Config.java @@ -0,0 +1,161 @@ +package com.teragrep.cfe_39; + +import org.apache.logging.log4j.core.config.ConfigurationSource; +import org.apache.logging.log4j.core.config.Configurator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.Enumeration; +import java.util.Properties; + +public class Config { + + // db + private final String dbConnectionUrl; + private final String dbUsername; + private final String dbPassword; + + // kafka + private final String queueTopicPattern; + + private final Properties kafkaConsumerProperties; + + private final boolean replicationEnabled; + private final int streamSize; + private final String streamUnits; + private static final Logger LOGGER = LoggerFactory.getLogger(Config.class); + private final int dropPartitionsOlderThanHours; + private final int createPartitionsInAdvanceHours; + private final boolean overrideTableLocation; + private final String tableLocation; + + Config() throws IOException { + Properties properties = new Properties(); + Path configPath = Paths.get(System.getProperty("cfe_30.config.location", System.getProperty("user.dir") + "/etc/application.properties")); + LOGGER.info("Loading application config '" + configPath.toAbsolutePath() + "'"); + properties.load(Files.newInputStream(configPath)); + LOGGER.debug("Got configuration: " + properties); + + // db + this.dbConnectionUrl = properties.getProperty("db.connectionUrl"); + if (this.dbConnectionUrl == null) { + throw new IllegalArgumentException("db.connectionUrl not set"); + } + this.dbUsername = properties.getProperty("db.username"); + if (this.dbUsername == null) { + throw new IllegalArgumentException("db.username not set"); + } + this.dbPassword = properties.getProperty("db.password"); + if (this.dbPassword == null) { + throw new IllegalArgumentException("db.password not set"); + } + + String replicationEnabledString = properties.getProperty("db.replicationEnabled", "false"); + this.replicationEnabled = Boolean.parseBoolean(replicationEnabledString); + + String streamBytesString = properties.getProperty("db.streamSize", "512"); + this.streamSize = Integer.parseInt(streamBytesString); + this.streamUnits = properties.getProperty("db.streamUnits", "rows"); + this.dropPartitionsOlderThanHours = Integer.parseInt(properties.getProperty("db.dropPartitionsOlderThanHours", "4")); + if(dropPartitionsOlderThanHours <= 0) { + throw new IllegalArgumentException("db.dropPartitionsOlderThanHours must be set to >0, got " + dropPartitionsOlderThanHours); + } + this.createPartitionsInAdvanceHours = Integer.parseInt(properties.getProperty("db.createPartitionsInAdvanceHours", "8")); + if(createPartitionsInAdvanceHours <= 0) { + throw new IllegalArgumentException("createPartitionsInAdvanceHours must be set to >0, got " + createPartitionsInAdvanceHours); + } + + String overrideTableLocationString = properties.getProperty("db.overrideTableLocation", "false"); + this.overrideTableLocation = Boolean.parseBoolean(overrideTableLocationString); + this.tableLocation = properties.getProperty("db.tableLocation"); + if(overrideTableLocation && tableLocation.isEmpty()) { + throw new IllegalArgumentException("db.tableLocation resulted in empty string when db.overrideTableLocation was true"); + } + + // kafka + this.queueTopicPattern = properties.getProperty("queueTopicPattern", "^.*$"); + + this.kafkaConsumerProperties = loadSubProperties(properties, "consumer."); + String loginConfig = properties.getProperty("java.security.auth.login.config"); + if(loginConfig == null) { + throw new IOException("Property java.security.auth.login.config does not exist"); + } + if(!(new File(loginConfig)).isFile()) { + throw new IOException("File '" + loginConfig + "' set by java.security.auth.login.config does not exist"); + } + System.setProperty("java.security.auth.login.config", loginConfig); + + // Just for loggers to work + Path log4j2Config = Paths.get(properties.getProperty("log4j2.configurationFile", System.getProperty("user.dir") + "/etc/log4j2.properties")); + LOGGER.info("Loading log4j2 config from '" + log4j2Config.toRealPath() + "'"); + Configurator.reconfigure(log4j2Config.toUri()); + } + + private Properties loadSubProperties(Properties properties, String prefix) { + Properties subProperties = new Properties(); + + Enumeration keys = properties.keys(); + while (keys.hasMoreElements()) { + String key = String.valueOf(keys.nextElement()); + if (key.startsWith(prefix)) { + String value = properties.getProperty(key); + String subKey = key.replaceFirst(prefix,""); + subProperties.put(subKey, value); + } + } + return subProperties; + } + + public boolean isOverrideTableLocation() { + return overrideTableLocation; + } + + public String getTableLocation() { + return tableLocation; + } + + public String getDbConnectionUrl() { + return dbConnectionUrl; + } + + public String getDbUsername() { + return dbUsername; + } + + public String getDbPassword() { + return dbPassword; + } + + public boolean isReplicationEnabled() { + return replicationEnabled; + } + + public int getStreamSize() { + return streamSize; + } + + public String getStreamUnits() { + return streamUnits; + } + + public String getQueueTopicPattern() { + return queueTopicPattern; + } + + public Properties getKafkaConsumerProperties() { + return kafkaConsumerProperties; + } + + public int getDropPartitionsOlderThanHours() { + return dropPartitionsOlderThanHours; + } + + public int getCreatePartitionsInAdvanceHours() { + return createPartitionsInAdvanceHours; + } +} \ No newline at end of file diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java new file mode 100644 index 00000000..7882e7e1 --- /dev/null +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java @@ -0,0 +1,117 @@ +package com.teragrep.cfe_39.consumers.kafka; + +import com.teragrep.cfe_39.Config; +import com.teragrep.cfe_39.metrics.topic.TopicCounter; +import com.teragrep.cfe_39.metrics.RuntimeStatistics; +import com.teragrep.rlo_06.*; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.sql.SQLException; +import java.time.Instant; +import java.util.List; +import java.util.function.Consumer; + +// TODO: +// Alter the class so the output is not SQL (aka. mariadb). +// Instead the kafka stream should first be deserialized using rlo_06 and then serialized using avro and stored in HDFS. +// The kafka offsets must be passed to HDFS too. + +public class DatabaseOutput implements Consumer> { + private static final Logger LOGGER = LoggerFactory.getLogger(DatabaseOutput.class); + private final RFC5424Frame rfc5424Frame = new RFC5424Frame(false); + + private final String table; + + + private final RuntimeStatistics runtimeStatistics; + private final TopicCounter topicCounter; + + private long lastTimeCalled = Instant.now().toEpochMilli(); + + public static final String ANSI_RESET = "\u001B[0m"; + public static final String ANSI_BLACK = "\u001B[30m"; + public static final String ANSI_RED = "\u001B[31m"; + public static final String ANSI_GREEN = "\u001B[32m"; + public static final String ANSI_YELLOW = "\u001B[33m"; + public static final String ANSI_BLUE = "\u001B[34m"; + public static final String ANSI_PURPLE = "\u001B[35m"; + public static final String ANSI_CYAN = "\u001B[36m"; + public static final String ANSI_WHITE = "\u001B[37m"; + + DatabaseOutput( + Config config, + String table, + RuntimeStatistics runtimeStatistics, + TopicCounter topicCounter + ) { + this.table = table; + this.runtimeStatistics = runtimeStatistics; + this.topicCounter = topicCounter; + } + + @Override + public void accept(List bytes) { + long thisTime = Instant.now().toEpochMilli(); + long ftook = thisTime - lastTimeCalled; + topicCounter.setKafkaLatency(ftook); + LOGGER.debug(ANSI_BLUE + "Fuura searching your batch for <[" + table + "]> with records <" + bytes.size() + "> and took <" + (ftook) + "> milliseconds. <" + (bytes.size() * 1000L / ftook) + "> EPS. " + ANSI_RESET); + long batchBytes = 0L; + + for (byte[] byteArray : bytes) { + batchBytes = batchBytes + byteArray.length; + InputStream inputStream = new ByteArrayInputStream(byteArray); + rfc5424Frame.load(inputStream); + try { + if(rfc5424Frame.next()) { + // TODO: rfc5424Frame has loaded the data, it's ready for deserialization. + new RFC5424Timestamp(rfc5424Frame.timestamp).toZonedDateTime().toInstant().getEpochSecond(); + rfc5424Frame.appName.toString(); + rfc5424Frame.hostname.toString(); + rfc5424Frame.msg.toString(); + } + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + long start = Instant.now().toEpochMilli(); + + // TODO: Add the code for sending the data to HDFS here, performance is measured between the start/end. + + long end = Instant.now().toEpochMilli(); + + long took = (end - start); + topicCounter.setDatabaseLatency(took); + + if (took == 0) { + took = 1; + } + long rps = bytes.size() * 1000L / took; + topicCounter.setRecordsPerSecond(rps); + + long bps = batchBytes * 1000 / took; + topicCounter.setBytesPerSecond(bps); + + runtimeStatistics.addAndGetRecords(bytes.size()); + runtimeStatistics.addAndGetBytes(batchBytes); + + topicCounter.addToTotalBytes(batchBytes); + topicCounter.addToTotalRecords(bytes.size()); + + LOGGER.debug( + ANSI_GREEN + + "Sent batch for <[" + table + "]> " + + "with records <" + bytes.size() + "> " + + "and size <" + batchBytes / 1024 + "> KB " + + "took <" + (took) + "> milliseconds. " + + "<" + rps + "> RPS. " + + "<" + bps / 1024 + "> KB/s " + + ANSI_RESET + ); + lastTimeCalled = Instant.now().toEpochMilli(); + } +} \ No newline at end of file diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java new file mode 100644 index 00000000..1b4dda92 --- /dev/null +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java @@ -0,0 +1,159 @@ +package com.teragrep.cfe_39.consumers.kafka; + + +import com.teragrep.cfe_39.Config; +import com.teragrep.cfe_39.metrics.*; +import com.teragrep.cfe_39.metrics.topic.TopicCounter; +import com.teragrep.cfe_39.metrics.topic.TopicStatistics; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +import java.sql.SQLException; +import java.time.Duration; +import java.util.*; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.function.Consumer; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +public class KafkaController { + // TODO: Create kafka consumer groups. + // rlo_09 KafkaReader is the code that I should take a look at. ReadCoordinator alone won't allow access to the kafka offsets, it must be done in KafkaReader. + // ReadCoordinator uses the KafkaReader, but it's set as private and there are no functions for accessing it through ReadCoordinator. + // The enable.auto.commit=false is set in config and it is fetched by the config.getKafkaConsumerProperties(). + // cfe_30 is already using enable.auto.commit=false, so looking through cfe_30 and rlo_09 ReadCoordinator and KafkaReader functions should get the coding on right track. + + // A consumer may opt to commit offsets by itself (enable.auto.commit=false). + // Depending on when it chooses to commit offsets, there are delivery semantics available to the consumer. + // Exactly once: + // - For Kafka topic to External System workflows, to effectively achieve exactly once, you must use an idempotent consumer. + + // An Idempotent Consumer pattern uses a Kafka consumer that can consume the same message any number of times, but only process it once. + // To implement the Idempotent Consumer pattern the recommended approach is to add a table to the database to track processed messages. + // Each message needs to have a unique messageId assigned by the producing service, either within the payload, or as a Kafka message header. + // When a new message is consumed the table is checked for the existence of this message Id. If present, then the message is a duplicate. + // The consumer updates its offsets to effectively mark the message as consumed to ensure it is not redelivered, and no further action takes place. + // If the message Id is not present in the table then a database transaction is started and the message Id is inserted. + // The message is then processed performing the required business logic. Upon completion the transaction is committed. + + // TODO requirements: + // 1. The KafkaReader-class must be able to pass the offset values to the main class that is calling KafkaReader. + // 2. The main class must handle the consuming of the kafka topics in an idempotent way as stated above. This is achieved by using the HDFS filenames to store the topic_name and offset values of Kafka topics. + + private static final Logger LOGGER = LoggerFactory.getLogger(KafkaController.class); + + private final Config config; + + private final org.apache.kafka.clients.consumer.Consumer kafkaConsumer; + private final List threads = new ArrayList<>(); + private final Set activeTopics = new HashSet<>(); + private final RuntimeStatistics runtimeStatistics = new RuntimeStatistics(); + + public KafkaController(Config config) { + this.config = config; + this.kafkaConsumer = new KafkaConsumer<>(config.getKafkaConsumerProperties(), new ByteArrayDeserializer(), new ByteArrayDeserializer()); + } + + public void run() throws InterruptedException { + // register runtime statistics + runtimeStatistics.register(); + + // register duration statistics + DurationStatistics durationStatistics = new DurationStatistics(); + durationStatistics.register(); + + // register per topic counting + List topicCounters = new CopyOnWriteArrayList<>(); + TopicStatistics topicMetrics = new TopicStatistics( + topicCounters + ); + topicMetrics.register(); + + while (true) { + LOGGER.debug("Scanning for threads"); + topicScan(durationStatistics, topicCounters); + + // log stuff + durationStatistics.log(); + long topicScanDelay = 30000L; + Thread.sleep(topicScanDelay); + } + } + + // Creates kafka topic consumer based on input parameters. + private void createReader(String topic, List topicCounters) throws SQLException { + TopicCounter topicCounter = new TopicCounter(topic); + topicCounters.add(topicCounter); + + Consumer> output = new DatabaseOutput( + config, + topic, + runtimeStatistics, + topicCounter + ); + + /* + Consumer> output = bytes -> { + for (byte[] bs : bytes) { + System.out.println(new String(bs, StandardCharsets.UTF_8)); + topicCounter.addToTotalRecords(1); + topicCounter.addToTotalBytes(new String(bs, StandardCharsets.UTF_8).length()); + } + }; +{1} + */ + + // The kafka offsets must be passed to HDFS. The consumer must also be set to manual commits so the HDFS can handle managing the commit offsets within the HDFS filenames. + // Instead of ReadCoordinator the KafkaReader must be used, as the coordinator won't give access to offsets. + ReadCoordinatorTemp readCoordinator = new ReadCoordinatorTemp( + topic, + config.getKafkaConsumerProperties(), + output + ); + + Thread readThread = new Thread(null, readCoordinator, topic); + threads.add(readThread); + readThread.start(); + } + + private void topicScan(DurationStatistics durationStatistics, List topicCounters) { + Map> listTopics = kafkaConsumer.listTopics(Duration.ofSeconds(60)); + Pattern topicsRegex = Pattern.compile(config.getQueueTopicPattern()); + + // Find the topics available in Kafka based on given QueueTopicPattern, both active and in-active. + // TODO: check how partitions are handled, need to allow using consumer groups for partition read assignments. aka. load balancing + Set foundTopics = new HashSet<>(); + for (Map.Entry> entry : listTopics.entrySet()) { + Matcher matcher = topicsRegex.matcher(entry.getKey()); + if (matcher.matches()) { + foundTopics.add(entry.getKey()); + } + } + + if (foundTopics.isEmpty()) { + throw new IllegalStateException("Pattern <[" + config.getQueueTopicPattern() + "]> found no topics." ); + } + + // subtract currently active topics from found topics + foundTopics.removeAll(activeTopics); + + // Activate all the found in-active topics, in other words create individual consumers for all of them using the createReader()-function. + for (String topic : foundTopics) { + LOGGER.info("Activating topic <"+topic+">"); + try { + createReader(topic, topicCounters); + activeTopics.add(topic); + runtimeStatistics.addAndGetThreads(1); + } + catch (SQLException sqlException) { + LOGGER.error("Topic <"+topic+"> not activated due to reader creation error: " + sqlException); + } + } + durationStatistics.report(runtimeStatistics); + } + +} diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReaderTemp.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReaderTemp.java new file mode 100644 index 00000000..ea723bbf --- /dev/null +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReaderTemp.java @@ -0,0 +1,63 @@ +package com.teragrep.cfe_39.consumers.kafka; + +import org.apache.kafka.clients.consumer.*; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; + +public class KafkaReaderTemp implements AutoCloseable { + final Logger LOGGER = LoggerFactory.getLogger(KafkaReaderTemp.class); + + private Iterator> kafkaRecordsIterator = Collections.emptyIterator(); + + private final Consumer kafkaConsumer; + + private final java.util.function.Consumer> callbackFunction; + + public KafkaReaderTemp( + Consumer kafkaConsumer, java.util.function.Consumer> callbackFunction) { + this.kafkaConsumer = kafkaConsumer; + this.callbackFunction = callbackFunction; + } + + public void read() { + long offset; + if (!kafkaRecordsIterator.hasNext()) { + // still need to consume more, infinitely loop because connection problems may cause return of an empty iterator + ConsumerRecords kafkaRecords = kafkaConsumer.poll(Duration.ofSeconds(60)); // TODO parametrize + if (kafkaRecords.isEmpty()) { + LOGGER.debug("kafkaRecords empty after poll."); + } + kafkaRecordsIterator = kafkaRecords.iterator(); + } + + List recordsList = new ArrayList<>(); + while (kafkaRecordsIterator.hasNext()) { + ConsumerRecord record = kafkaRecordsIterator.next(); + offset = record.offset(); // offset for HDFS filenames + LOGGER.debug("adding from offset: " + record.offset()); // TODO: The offsets must be passed outside the class, to the main class that is calling this class. + recordsList.add(record.value()); + } + + if (!recordsList.isEmpty()) { + callbackFunction.accept(recordsList); + kafkaConsumer.commitSync(); + /* + commitSync(): + It only commits the offsets that were actually polled and processed. If some offsets were not included in the last poll, then those offsets will not be committed. + It will not commit the latest positions for all subscribed partitions. This would interfere with the Consumer Offset management concept of Kafka to be able to re-start an application where it left off. + * */ + } + } + + @Override + public void close() { + kafkaConsumer.close(Duration.ofSeconds(60)); // TODO parametrize + } +} diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java new file mode 100644 index 00000000..9184b546 --- /dev/null +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java @@ -0,0 +1,163 @@ +package com.teragrep.cfe_39.consumers.kafka; + +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.MockConsumer; +import org.apache.kafka.clients.consumer.OffsetResetStrategy; +import org.apache.kafka.common.TopicPartition; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.nio.charset.StandardCharsets; +import java.util.Collections; +import java.util.HashMap; + +/** + *

Mock Kafka Consumer Factory

+ * + * Mocked Kafka Consumer factory used for testing. + * + * @since 08/06/2022 + * @author Mikko Kortelainen + */ +public class MockKafkaConsumerFactoryTemp { + final static private Logger LOGGER = LoggerFactory.getLogger(MockKafkaConsumerFactoryTemp.class); + + private MockKafkaConsumerFactoryTemp() { + + } + + private static void generateEvents(MockConsumer consumer) { + consumer.addRecord(new ConsumerRecord<>("testConsumerTopic", + 0, + 0L, + "2022-04-25T07:34:50.804Z".getBytes(StandardCharsets.UTF_8), + "<12>1 2022-04-25T07:34:50.804Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"835bf792-91cf-44e3-976b-518330bb8fd3\" source=\"source\" unixtime=\"1650872090805\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872090\"] [WARN] 2022-04-25 07:34:50,804 com.teragrep.jla_02.Log4j Log - Log4j warn says hi!".getBytes(StandardCharsets.UTF_8) + ) + ); + consumer.addRecord(new ConsumerRecord<>("testConsumerTopic", + 0, + 1L, + "2022-04-25T07:34:50.806Z".getBytes(StandardCharsets.UTF_8), + "<12>1 2022-04-25T07:34:50.806Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"c3f13f9a-05e2-41bd-b0ad-1eca6fd6fd9a\" source=\"source\" unixtime=\"1650872090806\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872090\"] [ERROR] 2022-04-25 07:34:50,806 com.teragrep.jla_02.Log4j Log - Log4j error says hi!".getBytes(StandardCharsets.UTF_8) + ) + ); + consumer.addRecord(new ConsumerRecord<>("testConsumerTopic", + 0, + 2L, + "2022-04-25T07:34:50.822Z".getBytes(StandardCharsets.UTF_8), + "<12>1 2022-04-25T07:34:50.822Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02\"][event_id@48577 hostname=\"jla-02\" uuid=\"1848d8a1-2f08-4a1e-bec4-ff9e6dd92553\" source=\"source\" unixtime=\"1650872090822\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872090\"] 470647 [Thread-3] INFO com.teragrep.jla_02.Logback Daily - Logback-daily says hi.".getBytes(StandardCharsets.UTF_8) + ) + ); + + consumer.addRecord(new ConsumerRecord<>("testConsumerTopic", + 0, + 3L, + "2022-04-25T07:34:50.822Z".getBytes(StandardCharsets.UTF_8), + "<12>1 2022-04-25T07:34:50.822Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02\"][event_id@48577 hostname=\"jla-02\" uuid=\"5e1a0398-c2a0-468d-a562-c3bb31f0f853\" source=\"source\" unixtime=\"1650872090822\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872090\"] 470646 [Thread-3] INFO com.teragrep.jla_02.Logback Audit - Logback-audit says hi.".getBytes(StandardCharsets.UTF_8) + ) + ); + + consumer.addRecord(new ConsumerRecord<>("testConsumerTopic", + 0, + 4L, + "2022-04-25T07:34:50.822Z".getBytes(StandardCharsets.UTF_8), + "<12>1 2022-04-25T07:34:50.822Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02\"][event_id@48577 hostname=\"jla-02\" uuid=\"6268c3a2-5bda-427f-acce-29416eb445f4\" source=\"source\" unixtime=\"1650872090822\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872090\"] 470647 [Thread-3] INFO com.teragrep.jla_02.Logback Metric - Logback-metric says hi.".getBytes(StandardCharsets.UTF_8) + ) + ); + + consumer.addRecord(new ConsumerRecord<>("testConsumerTopic", + 0, + 5L, + "2022-04-25T07:34:52.238Z".getBytes(StandardCharsets.UTF_8), + "<12>1 2022-04-25T07:34:52.238Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"b500dcaf-1101-4000-b6b9-bfb052ddbf86\" source=\"source\" unixtime=\"1650872092238\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.238 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info audit says hi!]".getBytes(StandardCharsets.UTF_8) + ) + ); + + consumer.addRecord(new ConsumerRecord<>("testConsumerTopic", + 0, + 6L, + "2022-04-25T07:34:52.239Z".getBytes(StandardCharsets.UTF_8), + "<12>1 2022-04-25T07:34:52.239Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"05363122-51ac-4c0b-a681-f5868081f56d\" source=\"source\" unixtime=\"1650872092239\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info daily says hi!]".getBytes(StandardCharsets.UTF_8) + ) + ); + + consumer.addRecord(new ConsumerRecord<>("testConsumerTopic", + 0, + 7L, + "2022-04-25T07:34:52.239Z".getBytes(StandardCharsets.UTF_8), + "<12>1 2022-04-25T07:34:52.239Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"7bbcd843-b795-4c14-b4a1-95f5d445cbcd\" source=\"source\" unixtime=\"1650872092239\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info metric says hi!]".getBytes(StandardCharsets.UTF_8) + ) + ); + + consumer.addRecord(new ConsumerRecord<>("testConsumerTopic", + 0, + 8L, + "2022-04-25T07:34:52.240Z".getBytes(StandardCharsets.UTF_8), + "<12>1 2022-04-25T07:34:52.240Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"2bc0a9f9-237d-4656-b40a-3038aace37f0\" source=\"source\" unixtime=\"1650872092240\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn audit says hi!]".getBytes(StandardCharsets.UTF_8) + ) + ); + + consumer.addRecord(new ConsumerRecord<>("testConsumerTopic", + 0, + 9L, + "2022-04-25T07:34:52.240Z".getBytes(StandardCharsets.UTF_8), + "<12>1 2022-04-25T07:34:52.240Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"ecf61e8d-e3a7-48ef-9b73-3c5a5243d2e6\" source=\"source\" unixtime=\"1650872092240\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn daily says hi!]".getBytes(StandardCharsets.UTF_8) + ) + ); + + consumer.addRecord(new ConsumerRecord<>("testConsumerTopic", + 0, + 10L, + "2022-04-25T07:34:52.241Z".getBytes(StandardCharsets.UTF_8), + "<12>1 2022-04-25T07:34:52.241Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"bf101d5a-e816-4f51-b132-97f8e3431f8e\" source=\"source\" unixtime=\"1650872092241\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.241 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn metric says hi!]".getBytes(StandardCharsets.UTF_8) + ) + ); + + consumer.addRecord(new ConsumerRecord<>("testConsumerTopic", + 0, + 11L, + "2022-04-25T07:34:52.241Z".getBytes(StandardCharsets.UTF_8), + "<12>1 2022-04-25T07:34:52.241Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"ef94d9e9-3c44-4892-b5a6-bf361d13ff97\" source=\"source\" unixtime=\"1650872092241\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.241 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error audit says hi!]".getBytes(StandardCharsets.UTF_8) + ) + ); + + consumer.addRecord(new ConsumerRecord<>("testConsumerTopic", + 0, + 12L, + "2022-04-25T07:34:52.242Z".getBytes(StandardCharsets.UTF_8), + "<12>1 2022-04-25T07:34:52.242Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"5bce6e3d-767d-44b4-a044-6c4872f8f2b5\" source=\"source\" unixtime=\"1650872092242\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.242 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error daily says hi!]".getBytes(StandardCharsets.UTF_8) + ) + ); + + consumer.addRecord(new ConsumerRecord<>("testConsumerTopic", + 0, + 13L, + "2022-04-25T07:34:52.243Z".getBytes(StandardCharsets.UTF_8), + "<12>1 2022-04-25T07:34:52.243Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"3bb55ce4-0ea7-413a-b403-28b174d7ac99\" source=\"source\" unixtime=\"1650872092243\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.243 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error metric says hi!]".getBytes(StandardCharsets.UTF_8) + ) + ); + } + + public static Consumer getConsumer() { + LOGGER.warn("useMockKafkaConsumer is set, using MockKafkaConsumer"); + final MockConsumer consumer; + + final TopicPartition topicPartition = new TopicPartition("testConsumerTopic", 0); + + consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); + consumer.assign(Collections.singletonList(topicPartition)); + + HashMap beginningOffsets = new HashMap<>(); + beginningOffsets.put(topicPartition, 0L); // start is inclusive + consumer.updateBeginningOffsets(beginningOffsets); + + generateEvents(consumer); + + HashMap endOffsets = new HashMap<>(); + endOffsets.put(topicPartition, 14L); // end is exclusive + consumer.updateEndOffsets(endOffsets); + LOGGER.debug(endOffsets.toString()); + return consumer; + } +} \ No newline at end of file diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinatorTemp.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinatorTemp.java new file mode 100644 index 00000000..9091435b --- /dev/null +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinatorTemp.java @@ -0,0 +1,71 @@ +package com.teragrep.cfe_39.consumers.kafka; + +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.*; +import java.util.function.Consumer; + +public class ReadCoordinatorTemp implements Runnable { + private static final Logger LOGGER = + LoggerFactory.getLogger(ReadCoordinatorTemp.class); + + private final String queueTopic; + private final Properties readerKafkaProperties; + private final Consumer> callbackFunction; + private boolean run = true; + + public ReadCoordinatorTemp( + String queueTopic, + Properties readerKafkaProperties, + Consumer> callbackFunction) + { + this.queueTopic = queueTopic; + this.readerKafkaProperties = readerKafkaProperties; + this.callbackFunction = callbackFunction; + } + + private KafkaReaderTemp createKafkaReader(Properties readerKafkaProperties, + String topic, + Consumer> callbackFunction, + boolean useMockKafkaConsumer) { + + org.apache.kafka.clients.consumer.Consumer kafkaConsumer; + if (useMockKafkaConsumer) { + kafkaConsumer = MockKafkaConsumerFactoryTemp.getConsumer(); + } else { + kafkaConsumer = new KafkaConsumer<>(readerKafkaProperties, new ByteArrayDeserializer(), new ByteArrayDeserializer()); + kafkaConsumer.subscribe(Collections.singletonList(topic)); + } + + return new KafkaReaderTemp(kafkaConsumer, callbackFunction); + } + + @Override + public void run() { + boolean useMockKafkaConsumer = Boolean.parseBoolean( + readerKafkaProperties.getProperty("useMockKafkaConsumer", "false") + ); + + + try ( + KafkaReaderTemp kafkaReader = createKafkaReader( + readerKafkaProperties, + queueTopic, + callbackFunction, + useMockKafkaConsumer + ) + ) { + while (run) { + kafkaReader.read(); + } + } + } + + public void stop() { + run = false; + } +} \ No newline at end of file diff --git a/src/main/java/com/teragrep/cfe_39/metrics/DurationStatistics.java b/src/main/java/com/teragrep/cfe_39/metrics/DurationStatistics.java new file mode 100644 index 00000000..684c65d7 --- /dev/null +++ b/src/main/java/com/teragrep/cfe_39/metrics/DurationStatistics.java @@ -0,0 +1,81 @@ +package com.teragrep.cfe_39.metrics; + +import com.teragrep.cfe_39.metrics.mxj.AtomicLongMXJItem; +import com.teragrep.cfe_39.metrics.mxj.MXJBeanDynamizer; +import com.teragrep.cfe_39.metrics.mxj.MXJEndpoint; +import com.teragrep.cfe_39.metrics.mxj.MXJItem; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.management.DynamicMBean; +import java.time.Instant; +import java.util.ArrayList; +import java.util.List; + +public class DurationStatistics { + private static final Logger LOGGER = LoggerFactory.getLogger(DurationStatistics.class); + private final AtomicLongMXJItem samplingIntervalStat = new AtomicLongMXJItem("samplingInterval"); + private final AtomicLongMXJItem recordsPerSecondStat = new AtomicLongMXJItem("recordsPerSecond"); + private final AtomicLongMXJItem bytesPerSecondStat = new AtomicLongMXJItem("bytesPerSecond"); + private final AtomicLongMXJItem bytes = new AtomicLongMXJItem("bytes"); + private final AtomicLongMXJItem records = new AtomicLongMXJItem("records"); + private Instant lastReportTime = Instant.now(); + private long lastBytes = 0L; + private long lastRecords = 0L; + + public DynamicMBean register() { + List itemList = new ArrayList<>(); + itemList.add(samplingIntervalStat); + itemList.add(recordsPerSecondStat); + itemList.add(bytesPerSecondStat); + + MXJBeanDynamizer dynamizer = new MXJBeanDynamizer(itemList); + + MXJEndpoint mxjEndpoint = new MXJEndpoint( + "com.teragrep.cfe_39", + "Metrics", + "DurationTotals", + dynamizer.createDynamicMBean() + ); + + return mxjEndpoint.register(); + } + + public void report(RuntimeStatistics runtimeStatistics) { + long currentRecords = runtimeStatistics.addAndGetRecords(0); + long currentBytes = runtimeStatistics.addAndGetBytes(0); + + long durationRecords = currentRecords - lastRecords; + long durationBytes = currentBytes - lastBytes; + + records.getSetter().accept(durationRecords); + bytes.getSetter().accept(durationBytes); + + Instant currentTime = Instant.now(); + long took = currentTime.toEpochMilli() - lastReportTime.toEpochMilli(); + + samplingIntervalStat.getSetter().accept(took); + + if (took == 0) { + took = 1; + } + + recordsPerSecondStat.getSetter().accept(durationRecords * 1000L / took); + bytesPerSecondStat.getSetter().accept(durationBytes * 1000L / took); + + // persist + lastReportTime = currentTime; + lastRecords = currentRecords; + lastBytes = currentBytes; + } + + public void log() { + LOGGER.info( + "## TOTAL records <" + records.getReader().get() + "> " + + "and size <" + bytes.getReader().get() / 1024 + "> KB " + + "<" + recordsPerSecondStat.getReader().get() + "> RPS. " + + "<" + bytesPerSecondStat.getReader().get() / 1024 + "> KB/s " + + "during <" + samplingIntervalStat.getReader().get() + "> ms" + ); + } +} diff --git a/src/main/java/com/teragrep/cfe_39/metrics/RuntimeStatistics.java b/src/main/java/com/teragrep/cfe_39/metrics/RuntimeStatistics.java new file mode 100644 index 00000000..58a9e9eb --- /dev/null +++ b/src/main/java/com/teragrep/cfe_39/metrics/RuntimeStatistics.java @@ -0,0 +1,48 @@ +package com.teragrep.cfe_39.metrics; + + +import com.teragrep.cfe_39.metrics.mxj.AtomicLongMXJItem; +import com.teragrep.cfe_39.metrics.mxj.MXJBeanDynamizer; +import com.teragrep.cfe_39.metrics.mxj.MXJEndpoint; +import com.teragrep.cfe_39.metrics.mxj.MXJItem; + +import javax.management.DynamicMBean; +import java.util.ArrayList; +import java.util.List; + +public class RuntimeStatistics { + private final AtomicLongMXJItem threadsStat = new AtomicLongMXJItem("threads"); + private final AtomicLongMXJItem bytesStat = new AtomicLongMXJItem("bytes"); + private final AtomicLongMXJItem recordsStat = new AtomicLongMXJItem("records"); + + + public DynamicMBean register() { + List itemList = new ArrayList<>(); + itemList.add(threadsStat); + itemList.add(bytesStat); + itemList.add(recordsStat); + + MXJBeanDynamizer dynamizer = new MXJBeanDynamizer(itemList); + + MXJEndpoint mxjEndpoint = new MXJEndpoint( + "com.teragrep.cfe_39", + "Metrics", + "RuntimeTotals", + dynamizer.createDynamicMBean() + ); + + return mxjEndpoint.register(); + } + + public long addAndGetThreads(long delta) { + return threadsStat.getAtomicLong().addAndGet(delta); + } + + public long addAndGetBytes(long delta) { + return bytesStat.getAtomicLong().addAndGet(delta); + } + + public long addAndGetRecords(long delta) { + return recordsStat.getAtomicLong().addAndGet(delta); + } +} diff --git a/src/main/java/com/teragrep/cfe_39/metrics/mxj/AtomicLongMXJItem.java b/src/main/java/com/teragrep/cfe_39/metrics/mxj/AtomicLongMXJItem.java new file mode 100644 index 00000000..1f45caa8 --- /dev/null +++ b/src/main/java/com/teragrep/cfe_39/metrics/mxj/AtomicLongMXJItem.java @@ -0,0 +1,43 @@ +package com.teragrep.cfe_39.metrics.mxj; + + +import com.teragrep.cfe_39.metrics.mxj.MXJItem; + +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Consumer; +import java.util.function.Supplier; + +public class AtomicLongMXJItem implements MXJItem { + private final Class attributeType; + private final String name; + private final Supplier reader; + private final Consumer setter; + + private final AtomicLong atomicLong = new AtomicLong(); + public AtomicLongMXJItem(String name) { + this.attributeType = Long.class; + this.name = name; + this.reader = atomicLong::get; + this.setter = atomicLong::set; + } + + public Class getAttributeType() { + return attributeType; + } + + public String getName() { + return name; + } + + public Supplier getReader() { + return reader; + } + + public Consumer getSetter() { + return setter; + } + + public AtomicLong getAtomicLong() { + return atomicLong; + } +} diff --git a/src/main/java/com/teragrep/cfe_39/metrics/mxj/MXJBeanDynamizer.java b/src/main/java/com/teragrep/cfe_39/metrics/mxj/MXJBeanDynamizer.java new file mode 100644 index 00000000..b7e9db89 --- /dev/null +++ b/src/main/java/com/teragrep/cfe_39/metrics/mxj/MXJBeanDynamizer.java @@ -0,0 +1,29 @@ +package com.teragrep.cfe_39.metrics.mxj; + +import com.teragrep.mxj_01.DynamicBean; // TODO: Fix the dependency failing to load. + +import javax.management.DynamicMBean; +import java.util.List; + +public class MXJBeanDynamizer { + private final List mxjItems; + + public MXJBeanDynamizer(List mxjItems) { + this.mxjItems = mxjItems; + } + + public DynamicMBean createDynamicMBean() { + DynamicBean.Builder builder = DynamicBean.builder(); + + for (MXJItem a : mxjItems) { + builder = builder.withSimpleAttribute( + a.getAttributeType(), + a.getName(), + a.getReader(), + a.getSetter() + ); + } + + return builder.build(); + } +} diff --git a/src/main/java/com/teragrep/cfe_39/metrics/mxj/MXJEndpoint.java b/src/main/java/com/teragrep/cfe_39/metrics/mxj/MXJEndpoint.java new file mode 100644 index 00000000..43f46b7d --- /dev/null +++ b/src/main/java/com/teragrep/cfe_39/metrics/mxj/MXJEndpoint.java @@ -0,0 +1,40 @@ +package com.teragrep.cfe_39.metrics.mxj; + +import javax.management.*; +import java.lang.management.ManagementFactory; + +public class MXJEndpoint { + + private final String domain; + private final String type; + private final String name; + private final DynamicMBean dynamicMBean; + + public MXJEndpoint(String domain, String type, String name, DynamicMBean dynamicMBean) { + this.domain = domain; + this.type = type; + this.name = name; + this.dynamicMBean = dynamicMBean; + } + + public DynamicMBean register() { + MBeanServer mBeanServer = ManagementFactory.getPlatformMBeanServer(); + + try { + ObjectName objectName = new ObjectName( + domain + ":type=" + type + ",name=" + name + ); + + mBeanServer.registerMBean( + dynamicMBean, + objectName + ); + return dynamicMBean; + } catch (MalformedObjectNameException + | NotCompliantMBeanException + | InstanceAlreadyExistsException + | MBeanRegistrationException exception) { + throw new RuntimeException(exception); + } + } +} \ No newline at end of file diff --git a/src/main/java/com/teragrep/cfe_39/metrics/mxj/MXJItem.java b/src/main/java/com/teragrep/cfe_39/metrics/mxj/MXJItem.java new file mode 100644 index 00000000..b29be2e2 --- /dev/null +++ b/src/main/java/com/teragrep/cfe_39/metrics/mxj/MXJItem.java @@ -0,0 +1,14 @@ +package com.teragrep.cfe_39.metrics.mxj; + +import java.util.function.Consumer; +import java.util.function.Supplier; + +public interface MXJItem { + Class getAttributeType(); + + String getName(); + + Supplier getReader(); + + Consumer getSetter(); +} \ No newline at end of file diff --git a/src/main/java/com/teragrep/cfe_39/metrics/topic/TopicCounter.java b/src/main/java/com/teragrep/cfe_39/metrics/topic/TopicCounter.java new file mode 100644 index 00000000..828eadf5 --- /dev/null +++ b/src/main/java/com/teragrep/cfe_39/metrics/topic/TopicCounter.java @@ -0,0 +1,69 @@ +package com.teragrep.cfe_39.metrics.topic; + + +import java.util.concurrent.atomic.AtomicLong; + +public class TopicCounter { + private final String topicName; + private final AtomicLong totalRecords = new AtomicLong(); + private final AtomicLong totalBytes = new AtomicLong(); + private final AtomicLong recordsPerSecond = new AtomicLong(); + private final AtomicLong bytesPerSecond = new AtomicLong(); + private final AtomicLong kafkaLatency = new AtomicLong(); + private final AtomicLong databaseLatency = new AtomicLong(); + + public TopicCounter(String topicName) { + this.topicName = topicName; + } + public long getTotalRecords() { + return totalRecords.get(); + } + + public long getTotalBytes () { + return totalBytes.get(); + } + + public long getRecordsPerSecond() { + return recordsPerSecond.get(); + } + + public long getBytesPerSecond() { + return bytesPerSecond.get(); + } + + public String getTopicName() { + return topicName; + } + + public long getKafkaLatency() { + return kafkaLatency.get(); + } + + public long getDatabaseLatency() { + return databaseLatency.get(); + } + + public void addToTotalRecords(long incrementBy) { + totalRecords.addAndGet(incrementBy); + } + + public void addToTotalBytes(long incrementBy) { + totalBytes.addAndGet(incrementBy); + } + + public void setRecordsPerSecond(long rps) { + recordsPerSecond.set(rps); + } + + public void setBytesPerSecond(long bps) { + bytesPerSecond.set(bps); + } + + public void setKafkaLatency(long latency) { + kafkaLatency.set(latency); + } + + public void setDatabaseLatency(long latency) { + databaseLatency.set(latency); + } +} diff --git a/src/main/java/com/teragrep/cfe_39/metrics/topic/TopicStatistics.java b/src/main/java/com/teragrep/cfe_39/metrics/topic/TopicStatistics.java new file mode 100644 index 00000000..04ae7d76 --- /dev/null +++ b/src/main/java/com/teragrep/cfe_39/metrics/topic/TopicStatistics.java @@ -0,0 +1,59 @@ +package com.teragrep.cfe_39.metrics.topic; + +import com.teragrep.cfe_39.metrics.mxj.MXJEndpoint; +import com.teragrep.mxj_01.CompositeDataWriter; +import com.teragrep.mxj_01.DynamicBean; +import com.teragrep.mxj_01.TabularDataWriter; + +import javax.management.*; +import java.util.List; + +public class TopicStatistics { + private final List topicList; + + private final DynamicBean dynamicBean; + + public TopicStatistics(List topicList) { + this.topicList = topicList; + + // page + CompositeDataWriter topicCounterWriter = CompositeDataWriter.builder(TopicCounter.class) + .withTypeName("topic") + .withTypeDescription("Topic throughput") + .withSimpleAttribute("topicName", TopicCounter::getTopicName) + .withSimpleAttribute("totalRecords", TopicCounter::getTotalRecords) + .withSimpleAttribute("totalBytes", TopicCounter::getTotalBytes) + .withSimpleAttribute("recordsPerSecond", TopicCounter::getRecordsPerSecond) + .withSimpleAttribute("bytesPerSecond", TopicCounter::getBytesPerSecond) + .withSimpleAttribute("kafkaLatency", TopicCounter::getKafkaLatency) + .withSimpleAttribute("databaseLatency", TopicCounter::getDatabaseLatency) + .build(); + + // book + TabularDataWriter topicListWriter = TabularDataWriter.builder(TopicCounter.class) + .withTypeName("topics") + .withTypeDescription("Topics counted") + .withIndexName("topicName") + .withCompositeDataWriter(topicCounterWriter) + .build(); + + + this.dynamicBean = DynamicBean.builder() + .withTabularAttribute( + "TopicStatistics", + () -> topicList, + topicListWriter + ) + .build(); + } + + public DynamicMBean register() { + MXJEndpoint mxjEndpoint = new MXJEndpoint( + "com.teragrep.cfe_30", + "Metrics", + "Topic", + dynamicBean + ); + return mxjEndpoint.register(); + } +} From 692061e9253cb600c00f1947ea2da20305aecba7 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Fri, 8 Dec 2023 16:20:26 +0200 Subject: [PATCH 002/146] Implementing idempotent consumer pattern to the Kafka consumer groups (WIP). --- .../cfe_39/consumers/kafka/KafkaController.java | 16 ++++++++++++---- .../cfe_39/consumers/kafka/KafkaReaderTemp.java | 1 + .../consumers/kafka/ReadCoordinatorTemp.java | 1 + 3 files changed, 14 insertions(+), 4 deletions(-) diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java index 1b4dda92..26f5148a 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java @@ -41,8 +41,14 @@ public class KafkaController { // The message is then processed performing the required business logic. Upon completion the transaction is committed. // TODO requirements: - // 1. The KafkaReader-class must be able to pass the offset values to the main class that is calling KafkaReader. - // 2. The main class must handle the consuming of the kafka topics in an idempotent way as stated above. This is achieved by using the HDFS filenames to store the topic_name and offset values of Kafka topics. + + // 1. The KafkaReader-class must be able to pass the offset values alongside the consumed message to the main class that is calling KafkaReader. + + // 2. The main class must handle the consuming of the kafka topics in an idempotent way as stated above. + // This is achieved by using the HDFS filenames to store the topic_name and offset values of Kafka topics. + // In other words Kafka consumers will consume topics normally according to the offsets that Kafka stores internally, + // but the processor of the consumed messages will check the offsets of the messages that are consumed by kafka and + // processes ONLY those messages that have not already been processed based on the offset values stored in HDFS filenames. private static final Logger LOGGER = LoggerFactory.getLogger(KafkaController.class); @@ -88,7 +94,9 @@ public void run() throws InterruptedException { private void createReader(String topic, List topicCounters) throws SQLException { TopicCounter topicCounter = new TopicCounter(topic); topicCounters.add(topicCounter); - + // DatabaseOutput handles transferring the consumed data to storage (S3, mariadb, HDFS, etc.) + // Kafka offset tracking must be included here. + // Topic is figured out in topicScan and the offsets for the topic should be figured out here. Consumer> output = new DatabaseOutput( config, topic, @@ -108,7 +116,7 @@ private void createReader(String topic, List topicCounters) throws */ // The kafka offsets must be passed to HDFS. The consumer must also be set to manual commits so the HDFS can handle managing the commit offsets within the HDFS filenames. - // Instead of ReadCoordinator the KafkaReader must be used, as the coordinator won't give access to offsets. + // plain rlo_09.ReadCoordinator won't give access to offset values. Implementing custom rlo_09 code in the package to achieve offset access. ReadCoordinatorTemp readCoordinator = new ReadCoordinatorTemp( topic, config.getKafkaConsumerProperties(), diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReaderTemp.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReaderTemp.java index ea723bbf..190e79fe 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReaderTemp.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReaderTemp.java @@ -48,6 +48,7 @@ public void read() { if (!recordsList.isEmpty()) { callbackFunction.accept(recordsList); kafkaConsumer.commitSync(); + /* commitSync(): It only commits the offsets that were actually polled and processed. If some offsets were not included in the last poll, then those offsets will not be committed. diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinatorTemp.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinatorTemp.java index 9091435b..d3683462 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinatorTemp.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinatorTemp.java @@ -17,6 +17,7 @@ public class ReadCoordinatorTemp implements Runnable { private final Properties readerKafkaProperties; private final Consumer> callbackFunction; private boolean run = true; + private long offset; public ReadCoordinatorTemp( String queueTopic, From 888d1574a4285d46af8fd1e4761560d58a9c2a88 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Mon, 11 Dec 2023 17:21:56 +0200 Subject: [PATCH 003/146] Implementing idempotent consumer pattern to the Kafka consumer groups (WIP2). --- .../consumers/kafka/DatabaseOutput.java | 19 ++++++------ .../consumers/kafka/KafkaController.java | 29 +++++++----------- .../consumers/kafka/KafkaReaderTemp.java | 30 +++++++++++-------- .../consumers/kafka/ReadCoordinatorTemp.java | 6 ++-- .../consumers/kafka/RecordOffsetObject.java | 21 +++++++++++++ 5 files changed, 62 insertions(+), 43 deletions(-) create mode 100644 src/main/java/com/teragrep/cfe_39/consumers/kafka/RecordOffsetObject.java diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java index 7882e7e1..c0c65339 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java @@ -18,9 +18,9 @@ // TODO: // Alter the class so the output is not SQL (aka. mariadb). // Instead the kafka stream should first be deserialized using rlo_06 and then serialized using avro and stored in HDFS. -// The kafka offsets must be passed to HDFS too. +// The target where the record is stored in HDFS is based on the topic, partition and offset. ie. topic_name/0.123456 where offset is 123456 -public class DatabaseOutput implements Consumer> { +public class DatabaseOutput implements Consumer> { private static final Logger LOGGER = LoggerFactory.getLogger(DatabaseOutput.class); private final RFC5424Frame rfc5424Frame = new RFC5424Frame(false); @@ -54,14 +54,15 @@ public class DatabaseOutput implements Consumer> { } @Override - public void accept(List bytes) { + public void accept(List recordOffsetObjectList) { long thisTime = Instant.now().toEpochMilli(); long ftook = thisTime - lastTimeCalled; topicCounter.setKafkaLatency(ftook); - LOGGER.debug(ANSI_BLUE + "Fuura searching your batch for <[" + table + "]> with records <" + bytes.size() + "> and took <" + (ftook) + "> milliseconds. <" + (bytes.size() * 1000L / ftook) + "> EPS. " + ANSI_RESET); + LOGGER.debug(ANSI_BLUE + "Fuura searching your batch for <[" + table + "]> with records <" + recordOffsetObjectList.size() + "> and took <" + (ftook) + "> milliseconds. <" + (recordOffsetObjectList.size() * 1000L / ftook) + "> EPS. " + ANSI_RESET); long batchBytes = 0L; - for (byte[] byteArray : bytes) { + for (RecordOffsetObject recordOffsetObject : recordOffsetObjectList) { + byte[] byteArray = recordOffsetObject.record; batchBytes = batchBytes + byteArray.length; InputStream inputStream = new ByteArrayInputStream(byteArray); rfc5424Frame.load(inputStream); @@ -90,22 +91,22 @@ public void accept(List bytes) { if (took == 0) { took = 1; } - long rps = bytes.size() * 1000L / took; + long rps = recordOffsetObjectList.size() * 1000L / took; topicCounter.setRecordsPerSecond(rps); long bps = batchBytes * 1000 / took; topicCounter.setBytesPerSecond(bps); - runtimeStatistics.addAndGetRecords(bytes.size()); + runtimeStatistics.addAndGetRecords(recordOffsetObjectList.size()); runtimeStatistics.addAndGetBytes(batchBytes); topicCounter.addToTotalBytes(batchBytes); - topicCounter.addToTotalRecords(bytes.size()); + topicCounter.addToTotalRecords(recordOffsetObjectList.size()); LOGGER.debug( ANSI_GREEN + "Sent batch for <[" + table + "]> " - + "with records <" + bytes.size() + "> " + + "with records <" + recordOffsetObjectList.size() + "> " + "and size <" + batchBytes / 1024 + "> KB " + "took <" + (took) + "> milliseconds. " + "<" + rps + "> RPS. " diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java index 26f5148a..37caf00d 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java @@ -21,7 +21,6 @@ import java.util.regex.Pattern; public class KafkaController { - // TODO: Create kafka consumer groups. // rlo_09 KafkaReader is the code that I should take a look at. ReadCoordinator alone won't allow access to the kafka offsets, it must be done in KafkaReader. // ReadCoordinator uses the KafkaReader, but it's set as private and there are no functions for accessing it through ReadCoordinator. // The enable.auto.commit=false is set in config and it is fetched by the config.getKafkaConsumerProperties(). @@ -92,29 +91,22 @@ public void run() throws InterruptedException { // Creates kafka topic consumer based on input parameters. private void createReader(String topic, List topicCounters) throws SQLException { + + // Create a new topicCounter object for the topic that has not been added to topicCounters-list yet. TopicCounter topicCounter = new TopicCounter(topic); + // Add the new topicCounter object to the list. topicCounters.add(topicCounter); + // DatabaseOutput handles transferring the consumed data to storage (S3, mariadb, HDFS, etc.) // Kafka offset tracking must be included here. - // Topic is figured out in topicScan and the offsets for the topic should be figured out here. - Consumer> output = new DatabaseOutput( - config, - topic, - runtimeStatistics, - topicCounter + // Topic is figured out in topicScan so the offsets for the topic should be figured out here. + Consumer> output = new DatabaseOutput( + config, // Configuration settings + topic, // String, the name of the topic + runtimeStatistics, // RuntimeStatistics object from metrics + topicCounter // TopicCounter object from metrics ); - /* - Consumer> output = bytes -> { - for (byte[] bs : bytes) { - System.out.println(new String(bs, StandardCharsets.UTF_8)); - topicCounter.addToTotalRecords(1); - topicCounter.addToTotalBytes(new String(bs, StandardCharsets.UTF_8).length()); - } - }; -{1} - */ - // The kafka offsets must be passed to HDFS. The consumer must also be set to manual commits so the HDFS can handle managing the commit offsets within the HDFS filenames. // plain rlo_09.ReadCoordinator won't give access to offset values. Implementing custom rlo_09 code in the package to achieve offset access. ReadCoordinatorTemp readCoordinator = new ReadCoordinatorTemp( @@ -123,6 +115,7 @@ private void createReader(String topic, List topicCounters) throws output ); + // Every consumer is run in a separate thread. Thread readThread = new Thread(null, readCoordinator, topic); threads.add(readThread); readThread.start(); diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReaderTemp.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReaderTemp.java index 190e79fe..e10cfeb3 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReaderTemp.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReaderTemp.java @@ -6,10 +6,7 @@ import org.slf4j.LoggerFactory; import java.time.Duration; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Iterator; -import java.util.List; +import java.util.*; public class KafkaReaderTemp implements AutoCloseable { final Logger LOGGER = LoggerFactory.getLogger(KafkaReaderTemp.class); @@ -18,10 +15,10 @@ public class KafkaReaderTemp implements AutoCloseable { private final Consumer kafkaConsumer; - private final java.util.function.Consumer> callbackFunction; + private final java.util.function.Consumer> callbackFunction; public KafkaReaderTemp( - Consumer kafkaConsumer, java.util.function.Consumer> callbackFunction) { + Consumer kafkaConsumer, java.util.function.Consumer> callbackFunction) { this.kafkaConsumer = kafkaConsumer; this.callbackFunction = callbackFunction; } @@ -37,17 +34,24 @@ public void read() { kafkaRecordsIterator = kafkaRecords.iterator(); } - List recordsList = new ArrayList<>(); + List recordOffsetObjectList = new ArrayList<>(); while (kafkaRecordsIterator.hasNext()) { ConsumerRecord record = kafkaRecordsIterator.next(); - offset = record.offset(); // offset for HDFS filenames - LOGGER.debug("adding from offset: " + record.offset()); // TODO: The offsets must be passed outside the class, to the main class that is calling this class. - recordsList.add(record.value()); + LOGGER.debug("adding from offset: " + record.offset()); + // Do filtering here. + boolean checkStuff = checkIfProcessed(record.topic(), record.partition(), record.offset()); // TODO: Create checkIfProcessed method. Checks if the record has already been processed and stored in HDFS. + if (!checkStuff) { + recordOffsetObjectList.add(new RecordOffsetObject(record.topic(), record.partition(), record.offset(), record.value())); + }else{ + // TODO: The consumer should update its offsets to effectively mark the message as consumed to ensure it is not redelivered, and no further action takes place. + } } - if (!recordsList.isEmpty()) { - callbackFunction.accept(recordsList); - kafkaConsumer.commitSync(); + if (!recordOffsetObjectList.isEmpty()) { + // This is the DatabaseOutput.accept() function. This is where the idempotent consumer pattern should be implemented. + // Offset and other required data for HDFS storage are added to the input parameters of the accept() function which processes the consumed record. + callbackFunction.accept(recordOffsetObjectList); + kafkaConsumer.commitSync(); // FIXME: Should the commitSync() be moved outside of the if-brackets so the consumer could skip the already processed records properly? /* commitSync(): diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinatorTemp.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinatorTemp.java index d3683462..b5e4c0d0 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinatorTemp.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinatorTemp.java @@ -15,14 +15,14 @@ public class ReadCoordinatorTemp implements Runnable { private final String queueTopic; private final Properties readerKafkaProperties; - private final Consumer> callbackFunction; + private final Consumer> callbackFunction; private boolean run = true; private long offset; public ReadCoordinatorTemp( String queueTopic, Properties readerKafkaProperties, - Consumer> callbackFunction) + Consumer> callbackFunction) { this.queueTopic = queueTopic; this.readerKafkaProperties = readerKafkaProperties; @@ -31,7 +31,7 @@ public ReadCoordinatorTemp( private KafkaReaderTemp createKafkaReader(Properties readerKafkaProperties, String topic, - Consumer> callbackFunction, + Consumer> callbackFunction, boolean useMockKafkaConsumer) { org.apache.kafka.clients.consumer.Consumer kafkaConsumer; diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/RecordOffsetObject.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/RecordOffsetObject.java new file mode 100644 index 00000000..a7b27f3e --- /dev/null +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/RecordOffsetObject.java @@ -0,0 +1,21 @@ +package com.teragrep.cfe_39.consumers.kafka; + +// This is the class for handling the Kafka record topic/partition/offset data that are required for HDFS storage. +public class RecordOffsetObject { + public String topic; + public Integer partition; + public Long offset; + public byte[] record; + + public RecordOffsetObject( + String topic, + int partition, + long offset, + byte[] record + ) { + this.topic = topic; + this.partition = partition; + this.offset = offset; + this.record = record; + } +} From 2ab05a435cecb163d648f9d8919bc89622469cb5 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Tue, 12 Dec 2023 17:01:15 +0200 Subject: [PATCH 004/146] Implementing Kafka consumed record processing using AVRO serialization and HDFS storage (WIP). Skipping implementing idempotent consumer pattern to the Kafka consumer groups. --- pom.xml | 48 ++++++++-- src/main/avro/KafkaRecord.avsc | 7 ++ .../consumers/kafka/DatabaseOutput.java | 87 ++++++++++++++++--- .../consumers/kafka/KafkaController.java | 6 +- .../consumers/kafka/KafkaReaderTemp.java | 14 +-- .../consumers/kafka/SyslogAvroWriter.java | 59 +++++++++++++ 6 files changed, 195 insertions(+), 26 deletions(-) create mode 100644 src/main/avro/KafkaRecord.avsc create mode 100644 src/main/java/com/teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java diff --git a/pom.xml b/pom.xml index 6f9c7eec..2137b210 100644 --- a/pom.xml +++ b/pom.xml @@ -19,9 +19,40 @@ + + org.apache.avro + avro-maven-plugin + 1.11.3 + + + generate-sources + + schema + + + ${project.basedir}/src/main/avro/ + ${project.basedir}/src/main/java/ + + + + + + org.apache.maven.plugins + maven-compiler-plugin + + 1.8 + 1.8 + + + + com.teragrep rlo_06 @@ -32,11 +63,23 @@ rlo_09 2.0.4 + + + org.xerial.snappy + snappy-java + 1.1.10.5 + org.apache.avro avro 1.11.3 + + + org.apache.hadoop + hadoop-client + 3.3.6 + org.apache.logging.log4j log4j-slf4j2-impl @@ -47,11 +90,6 @@ log4j-core 2.19.0 - - org.apache.logging.log4j - log4j-slf4j2-impl - 2.20.0 - org.slf4j slf4j-api diff --git a/src/main/avro/KafkaRecord.avsc b/src/main/avro/KafkaRecord.avsc new file mode 100644 index 00000000..483ced9e --- /dev/null +++ b/src/main/avro/KafkaRecord.avsc @@ -0,0 +1,7 @@ +{"namespace": "com.teragrep.cfe_39.avro", + "type": "record", + "name": "SyslogRecord", + "fields": [ + {"name": "content", "type": "bytes"} + ] +} \ No newline at end of file diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java index c0c65339..b1d70028 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java @@ -1,13 +1,18 @@ package com.teragrep.cfe_39.consumers.kafka; import com.teragrep.cfe_39.Config; +import com.teragrep.cfe_39.avro.SyslogRecord; import com.teragrep.cfe_39.metrics.topic.TopicCounter; import com.teragrep.cfe_39.metrics.RuntimeStatistics; import com.teragrep.rlo_06.*; +import org.apache.avro.file.SyncableFileOutputStream; +import org.apache.hadoop.fs.*; +import org.apache.hadoop.hdfs.DistributedFileSystem; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.ByteArrayInputStream; +import java.io.File; import java.io.IOException; import java.io.InputStream; import java.sql.SQLException; @@ -15,10 +20,14 @@ import java.util.List; import java.util.function.Consumer; +import org.apache.hadoop.conf.Configuration; + +import java.net.URI; + // TODO: -// Alter the class so the output is not SQL (aka. mariadb). -// Instead the kafka stream should first be deserialized using rlo_06 and then serialized using avro and stored in HDFS. +// The kafka stream should first be deserialized using rlo_06 and then serialized again using avro and stored in HDFS. // The target where the record is stored in HDFS is based on the topic, partition and offset. ie. topic_name/0.123456 where offset is 123456 +// First implement the AVRO serialization (saves the data into a file) and then implement the HDFS access for storing the data. public class DatabaseOutput implements Consumer> { private static final Logger LOGGER = LoggerFactory.getLogger(DatabaseOutput.class); @@ -33,14 +42,12 @@ public class DatabaseOutput implements Consumer> { private long lastTimeCalled = Instant.now().toEpochMilli(); public static final String ANSI_RESET = "\u001B[0m"; - public static final String ANSI_BLACK = "\u001B[30m"; - public static final String ANSI_RED = "\u001B[31m"; public static final String ANSI_GREEN = "\u001B[32m"; - public static final String ANSI_YELLOW = "\u001B[33m"; public static final String ANSI_BLUE = "\u001B[34m"; - public static final String ANSI_PURPLE = "\u001B[35m"; - public static final String ANSI_CYAN = "\u001B[36m"; - public static final String ANSI_WHITE = "\u001B[37m"; + private SyslogAvroWriter syslogAvroWriter; + private final long minimumFreeSpace; + private final long maximumFileSize; + private final WritableQueue writableQueue; // TODO: Implement the WritableQueue for managing the stream of records. DatabaseOutput( Config config, @@ -51,6 +58,8 @@ public class DatabaseOutput implements Consumer> { this.table = table; this.runtimeStatistics = runtimeStatistics; this.topicCounter = topicCounter; + this.minimumFreeSpace = 32; // TODO: CHECK RIGHT VALUE + this.maximumFileSize = 64; } @Override @@ -61,14 +70,24 @@ public void accept(List recordOffsetObjectList) { LOGGER.debug(ANSI_BLUE + "Fuura searching your batch for <[" + table + "]> with records <" + recordOffsetObjectList.size() + "> and took <" + (ftook) + "> milliseconds. <" + (recordOffsetObjectList.size() * 1000L / ftook) + "> EPS. " + ANSI_RESET); long batchBytes = 0L; + // TODO: The recordOffsetObjectList loop will go through all the objects in the list. + // While it goes through the list, the contents of the objects are serialized into an AVRO-file. + // When the file size is going to go above 64M, commit the file into HDFS and start fresh with an empty AVRO-file. + // Serialize the object that was going to make the file go above 64M into the now empty AVRO-file and continue the loop. + // https://pagure.xnet.fi/com-teragrep/rlo_09/blob/avroness/f/src/main/java/com/teragrep/rlo_09/SyslogAvroWriter.java + // https://pagure.xnet.fi/com-teragrep/rlo_09/blob/avroness/f/src/main/java/com/teragrep/rlo_09/WriteCoordinator.java for (RecordOffsetObject recordOffsetObject : recordOffsetObjectList) { - byte[] byteArray = recordOffsetObject.record; + byte[] byteArray = recordOffsetObject.record; // loads the byte[] contained in recordOffsetObject.record to byteArray. batchBytes = batchBytes + byteArray.length; InputStream inputStream = new ByteArrayInputStream(byteArray); rfc5424Frame.load(inputStream); try { if(rfc5424Frame.next()) { - // TODO: rfc5424Frame has loaded the data, it's ready for deserialization. + // rfc5424Frame has loaded the data, it's ready for deserialization. + // Implement AVRO serialization for the Kafka records here, preparing the data for writing to HDFS. + // Write all the data into a file using AVRO. + // The size of each AVRO-serialized file should be as close to 64M as possible, and the name of the file should be set based on topic+partition+offset. + new RFC5424Timestamp(rfc5424Frame.timestamp).toZonedDateTime().toInstant().getEpochSecond(); rfc5424Frame.appName.toString(); rfc5424Frame.hostname.toString(); @@ -81,7 +100,53 @@ public void accept(List recordOffsetObjectList) { long start = Instant.now().toEpochMilli(); - // TODO: Add the code for sending the data to HDFS here, performance is measured between the start/end. + // Add the code for sending the AVRO-serialized data to HDFS here, performance is measured between the start/end. + // Also remember to implement Kerberized access to HDFS. + String hdfsuri = ""; // Get from config. + + String path="/user/hdfs/example/hdfs/"; + String fileName= recordOffsetObjectList+".csv"; + String fileContent="hello;world"; + + // ====== Init HDFS File System Object + Configuration conf = new Configuration(); + // Set FileSystem URI + conf.set("fs.defaultFS", hdfsuri); + // Because of Maven + conf.set("fs.hdfs.impl", DistributedFileSystem.class.getName()); + conf.set("fs.file.impl", LocalFileSystem.class.getName()); + // Set HADOOP user + System.setProperty("HADOOP_USER_NAME", "hdfs"); + System.setProperty("hadoop.home.dir", "/"); + //Get the filesystem - HDFS + try { + FileSystem fs = FileSystem.get(URI.create(hdfsuri), conf); + + //==== Create folder if not exists + Path workingDir=fs.getWorkingDirectory(); + Path newFolderPath= new Path(path); + if(!fs.exists(newFolderPath)) { + // Create new Directory + fs.mkdirs(newFolderPath); + // logger.info("Path "+path+" created."); + } + + //==== Write file + // logger.info("Begin Write file into hdfs"); + //Create a path + Path hdfswritepath = new Path(newFolderPath + "/" + fileName); + //Init output stream + FSDataOutputStream outputStream=fs.create(hdfswritepath); + //Cassical output stream usage + outputStream.writeBytes(fileContent); + outputStream.close(); + // logger.info("End Write file into hdfs"); + + } catch (IOException e) { + throw new RuntimeException(e); + } + + // TODO END long end = Instant.now().toEpochMilli(); diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java index 37caf00d..a250f28b 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java @@ -39,10 +39,8 @@ public class KafkaController { // If the message Id is not present in the table then a database transaction is started and the message Id is inserted. // The message is then processed performing the required business logic. Upon completion the transaction is committed. - // TODO requirements: - + // requirements: SKIPPING IDEMPOTENT IMPLEMENTATION FOR NOW! // 1. The KafkaReader-class must be able to pass the offset values alongside the consumed message to the main class that is calling KafkaReader. - // 2. The main class must handle the consuming of the kafka topics in an idempotent way as stated above. // This is achieved by using the HDFS filenames to store the topic_name and offset values of Kafka topics. // In other words Kafka consumers will consume topics normally according to the offsets that Kafka stores internally, @@ -126,7 +124,7 @@ private void topicScan(DurationStatistics durationStatistics, List Pattern topicsRegex = Pattern.compile(config.getQueueTopicPattern()); // Find the topics available in Kafka based on given QueueTopicPattern, both active and in-active. - // TODO: check how partitions are handled, need to allow using consumer groups for partition read assignments. aka. load balancing + // Check how partitions are handled, need to allow using consumer groups for partition read assignments. aka. load balancing Set foundTopics = new HashSet<>(); for (Map.Entry> entry : listTopics.entrySet()) { Matcher matcher = topicsRegex.matcher(entry.getKey()); diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReaderTemp.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReaderTemp.java index e10cfeb3..d853aea4 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReaderTemp.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReaderTemp.java @@ -38,20 +38,22 @@ public void read() { while (kafkaRecordsIterator.hasNext()) { ConsumerRecord record = kafkaRecordsIterator.next(); LOGGER.debug("adding from offset: " + record.offset()); - // Do filtering here. - boolean checkStuff = checkIfProcessed(record.topic(), record.partition(), record.offset()); // TODO: Create checkIfProcessed method. Checks if the record has already been processed and stored in HDFS. + recordOffsetObjectList.add(new RecordOffsetObject(record.topic(), record.partition(), record.offset(), record.value())); + +/* // SKIPPING IDEMPOTENT CONSUMER IMPLEMENTATION FOR NOW! + boolean checkStuff = checkIfProcessed(record.topic(), record.partition(), record.offset()); // Create checkIfProcessed method. Checks if the record has already been processed and stored in HDFS. if (!checkStuff) { recordOffsetObjectList.add(new RecordOffsetObject(record.topic(), record.partition(), record.offset(), record.value())); }else{ - // TODO: The consumer should update its offsets to effectively mark the message as consumed to ensure it is not redelivered, and no further action takes place. - } + // The consumer should update its offsets to effectively mark the message as consumed to ensure it is not redelivered, and no further action takes place. + }*/ } if (!recordOffsetObjectList.isEmpty()) { - // This is the DatabaseOutput.accept() function. This is where the idempotent consumer pattern should be implemented. + // This is the DatabaseOutput.accept() function. // Offset and other required data for HDFS storage are added to the input parameters of the accept() function which processes the consumed record. callbackFunction.accept(recordOffsetObjectList); - kafkaConsumer.commitSync(); // FIXME: Should the commitSync() be moved outside of the if-brackets so the consumer could skip the already processed records properly? + kafkaConsumer.commitSync(); /* commitSync(): diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java new file mode 100644 index 00000000..1c6f29ac --- /dev/null +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java @@ -0,0 +1,59 @@ +package com.teragrep.cfe_39.consumers.kafka; + +import com.teragrep.cfe_39.avro.SyslogRecord; +import org.apache.avro.file.CodecFactory; +import org.apache.avro.file.DataFileWriter; +import org.apache.avro.file.SeekableFileInput; +import org.apache.avro.file.SyncableFileOutputStream; +import org.apache.avro.io.DatumWriter; +import org.apache.avro.specific.SpecificDatumWriter; + +import java.io.*; +import java.nio.ByteBuffer; + +class SyslogAvroWriter implements AutoCloseable { + + private final DatumWriter datumWriter = + new SpecificDatumWriter<>(SyslogRecord.class); + + private final SyncableFileOutputStream syncableFileOutputStream; + + private final DataFileWriter dataFileWriter = new DataFileWriter<>(datumWriter); + + SyslogAvroWriter(File syslogFile) throws IOException { + dataFileWriter.setCodec(CodecFactory.snappyCodec()); + + + syncableFileOutputStream = + new SyncableFileOutputStream(syslogFile); + + syncableFileOutputStream.getChannel().tryLock(); + + if (syslogFile.length() == 0) { + // new file + dataFileWriter.create(SyslogRecord.getClassSchema(), syncableFileOutputStream); + } + else { + // existing file + SeekableFileInput seekableFileInput = new SeekableFileInput(syslogFile); + + // seek to end + syncableFileOutputStream.getChannel().position(syncableFileOutputStream.getChannel().size()); + dataFileWriter.appendTo(seekableFileInput, syncableFileOutputStream); + } + } + + + void write(SyslogRecord syslogRecord) throws IOException{ + dataFileWriter.append(syslogRecord); + dataFileWriter.flush(); // TODO: Avro files 'flush' must be called as few times as possible. Check memory usage impact + } + + public void close() throws IOException { + dataFileWriter.close(); + } + + public long getFileSize() throws IOException { + return syncableFileOutputStream.getChannel().size(); + } +} \ No newline at end of file From 9a8065c3a36a732905b22965c4e1fccce3e920b3 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Wed, 13 Dec 2023 15:57:05 +0200 Subject: [PATCH 005/146] Implementing AVRO serialization to Kafka record processing. --- .../consumers/kafka/DatabaseOutput.java | 87 ++++++++++++++++--- .../consumers/kafka/queue/QueueUtilities.java | 56 ++++++++++++ .../consumers/kafka/queue/WritableQueue.java | 58 +++++++++++++ 3 files changed, 190 insertions(+), 11 deletions(-) create mode 100644 src/main/java/com/teragrep/cfe_39/consumers/kafka/queue/QueueUtilities.java create mode 100644 src/main/java/com/teragrep/cfe_39/consumers/kafka/queue/WritableQueue.java diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java index b1d70028..a2bc9216 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java @@ -2,25 +2,22 @@ import com.teragrep.cfe_39.Config; import com.teragrep.cfe_39.avro.SyslogRecord; +import com.teragrep.cfe_39.consumers.kafka.queue.WritableQueue; import com.teragrep.cfe_39.metrics.topic.TopicCounter; import com.teragrep.cfe_39.metrics.RuntimeStatistics; import com.teragrep.rlo_06.*; -import org.apache.avro.file.SyncableFileOutputStream; import org.apache.hadoop.fs.*; import org.apache.hadoop.hdfs.DistributedFileSystem; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.ByteArrayInputStream; -import java.io.File; -import java.io.IOException; -import java.io.InputStream; -import java.sql.SQLException; +import java.io.*; import java.time.Instant; import java.util.List; import java.util.function.Consumer; import org.apache.hadoop.conf.Configuration; +import java.nio.ByteBuffer; import java.net.URI; @@ -44,6 +41,7 @@ public class DatabaseOutput implements Consumer> { public static final String ANSI_RESET = "\u001B[0m"; public static final String ANSI_GREEN = "\u001B[32m"; public static final String ANSI_BLUE = "\u001B[34m"; + private final SyslogRecord syslogRecord = new SyslogRecord(); private SyslogAvroWriter syslogAvroWriter; private final long minimumFreeSpace; private final long maximumFileSize; @@ -58,8 +56,34 @@ public class DatabaseOutput implements Consumer> { this.table = table; this.runtimeStatistics = runtimeStatistics; this.topicCounter = topicCounter; - this.minimumFreeSpace = 32; // TODO: CHECK RIGHT VALUE + this.minimumFreeSpace = 32; // TODO: CHECK RIGHT VALUE FOR minimumFreeSpace this.maximumFileSize = 64; + // TODO: Extract queueDirectory and queueNamePrefix values from config! + // queueDirectory (folder) is named based on topic name, something like this: hdfs:///opt/teragrep/cfe_39/srv/topic_name + // queueNamePrefix is partition, for example 0 + // The end result that is produced in WritableQueue is hdfs:///opt/teragrep/cfe_39/srv/topic_name/0.12345 where 12345 is offset and 0 the partition. + this.writableQueue = new WritableQueue( + "queueDirectory", + "queueNamePrefix" + ); + } + + boolean checkSizeTooLarge(long fileSize) { + try { + // If the syslogAvroWriter is already initialized, check the filesize so it doesn't go above 64M. + if (fileSize > maximumFileSize) { + // file too large + + syslogAvroWriter.close(); + File syslogFile = + writableQueue.getNextWritableFile(); + syslogAvroWriter = new SyslogAvroWriter(syslogFile); + return true; + } + } catch (IOException ioException) { + throw new UncheckedIOException(ioException); + } + return false; } @Override @@ -72,32 +96,73 @@ public void accept(List recordOffsetObjectList) { // TODO: The recordOffsetObjectList loop will go through all the objects in the list. // While it goes through the list, the contents of the objects are serialized into an AVRO-file. - // When the file size is going to go above 64M, commit the file into HDFS and start fresh with an empty AVRO-file. + // When the file size is about to go above 64M, commit the file into HDFS using the latest topic/partition/offset values as the filename and start fresh with an empty AVRO-file. // Serialize the object that was going to make the file go above 64M into the now empty AVRO-file and continue the loop. // https://pagure.xnet.fi/com-teragrep/rlo_09/blob/avroness/f/src/main/java/com/teragrep/rlo_09/SyslogAvroWriter.java // https://pagure.xnet.fi/com-teragrep/rlo_09/blob/avroness/f/src/main/java/com/teragrep/rlo_09/WriteCoordinator.java + // every recordOffsetObject.record on the recordOffsetObjectList basically represents a rlo_09 WriteCoordinator.accept(byte[] bytes) when the list is gone through in a loop. for (RecordOffsetObject recordOffsetObject : recordOffsetObjectList) { + + // Initializing syslogAvroWriter. + if (syslogAvroWriter == null) { + try { + File syslogFile = + writableQueue.getNextWritableFile(); + // TODO: Check how topic name, partition and offset should be added to the HDFS filename. + // The avro serialization filename shouldn't really matter as long as the name is changed when stuff is stored to HDFS. + // And the content of the AVRO-serialized file that is going to be stored in HDFS is finalized only when the maximumFileSize has been reached. + // This means the HDFS filename is only finalized when the AVRO-serialized file is finalized, because every Kafka-record added to the file is going to change the offset that is going to be used for the filename. + syslogAvroWriter = new SyslogAvroWriter(syslogFile); + } catch (IOException ioException) { + throw new IllegalArgumentException(ioException); + } + } else { + try { + checkSizeTooLarge(syslogAvroWriter.getFileSize()); + } catch (IOException ioException) { + throw new UncheckedIOException(ioException); + } + } + byte[] byteArray = recordOffsetObject.record; // loads the byte[] contained in recordOffsetObject.record to byteArray. batchBytes = batchBytes + byteArray.length; InputStream inputStream = new ByteArrayInputStream(byteArray); rfc5424Frame.load(inputStream); try { if(rfc5424Frame.next()) { - // rfc5424Frame has loaded the data, it's ready for deserialization. + // rfc5424Frame has loaded the record data, it's ready for deserialization. // Implement AVRO serialization for the Kafka records here, preparing the data for writing to HDFS. // Write all the data into a file using AVRO. // The size of each AVRO-serialized file should be as close to 64M as possible, and the name of the file should be set based on topic+partition+offset. - new RFC5424Timestamp(rfc5424Frame.timestamp).toZonedDateTime().toInstant().getEpochSecond(); + // TODO: Include stream (aka sourcetype) and directory (aka index) as well in syslogRecord. It only has record at the moment. Basically make the schema similar to that was used in mariadb... + syslogRecord.setContent(ByteBuffer.wrap(byteArray)); // byteArray is the loop's current recordOffsetObject.record + // Calculate the size of syslogRecord that is going to be written to syslogAvroWriter-file. + long capacity = syslogRecord.toByteBuffer().capacity(); + checkSizeTooLarge(syslogAvroWriter.getFileSize() + capacity); // TODO: Fix the checkSizeTooLarge functionality so the closed syslogAvroWriter-file can be transferred to HDFS. + syslogAvroWriter.write(syslogRecord); + + /*new RFC5424Timestamp(rfc5424Frame.timestamp).toZonedDateTime().toInstant().getEpochSecond(); rfc5424Frame.appName.toString(); rfc5424Frame.hostname.toString(); - rfc5424Frame.msg.toString(); + rfc5424Frame.msg.toString();*/ } } catch (IOException e) { throw new RuntimeException(e); } } + // Handle possible "leftover" syslogRecords from the loop. TODO: Most likely borked like this, fix it in testing. + try { + if (syslogAvroWriter.getFileSize() > 0) { + syslogAvroWriter.write(syslogRecord); + } + } catch (IOException e) { + throw new RuntimeException(e); + } + + // TODO: BELOW STUFF IS GOING TO BE EITHER SCRAPPED OR MOVED TO ANOTHER METHOD WHICH IS THEN CALLED SEPARATELY! EVERYTHING SHOULD BE DONE WITHIN THE ABOVE LOOP! + long start = Instant.now().toEpochMilli(); // Add the code for sending the AVRO-serialized data to HDFS here, performance is measured between the start/end. diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/queue/QueueUtilities.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/queue/QueueUtilities.java new file mode 100644 index 00000000..3501038c --- /dev/null +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/queue/QueueUtilities.java @@ -0,0 +1,56 @@ +package com.teragrep.cfe_39.consumers.kafka.queue; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.attribute.BasicFileAttributes; +import java.util.function.BiPredicate; +import java.util.function.ToLongFunction; + +class QueueUtilities { + private static final Logger LOGGER = + LoggerFactory.getLogger(QueueUtilities.class); + + static BiPredicate getFileMatcher(String queueNamePrefix) { + return (path, basicFileAttributes) -> { + if (!path.getFileName().toString().startsWith(queueNamePrefix)) { + return false; + } else if (path.getFileName().toString().endsWith(".state")) { + return false; + } else if (!basicFileAttributes.isRegularFile()) { + return false; + } else { + LOGGER.trace("getFileMatcher returning: " + path); + return true; + } + }; + } + + static void accessCheck(Path queueDirectory) { + if (!Files.isDirectory(queueDirectory)) { + throw new IllegalArgumentException( + "Provided path is not a " + + "directory <[" + queueDirectory + "]>"); + } + + if (!Files.isWritable(queueDirectory)) { + throw new IllegalArgumentException( + "Provided path is not " + + "writeable <[" + queueDirectory + "]>"); + } + } + + static ToLongFunction getPathToSequenceNumberFunction() { + return path -> { + String pathString = path.toString(); + + int dotPosition = pathString.lastIndexOf('.'); + + String sequenceNumberString = pathString.substring(dotPosition + 1); + + return Long.parseLong(sequenceNumberString); + }; + } +} \ No newline at end of file diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/queue/WritableQueue.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/queue/WritableQueue.java new file mode 100644 index 00000000..0225dbab --- /dev/null +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/queue/WritableQueue.java @@ -0,0 +1,58 @@ +package com.teragrep.cfe_39.consumers.kafka.queue; + +import java.io.File; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.nio.file.FileVisitOption; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.stream.Stream; + +public class WritableQueue { + private final Path queueDirectory; + private final String queueNamePrefix; + + public WritableQueue( + String queueDirectory, + String queueNamePrefix + ) { + this.queueDirectory = Paths.get(queueDirectory); + this.queueNamePrefix = queueNamePrefix; + QueueUtilities.accessCheck(this.queueDirectory); + } + + private File getNextWritableFilename() throws IOException { + + try (Stream files = Files.find( + queueDirectory, + 1, + QueueUtilities.getFileMatcher(queueNamePrefix), + FileVisitOption.FOLLOW_LINKS + )) { + + long sequenceNumber = files.mapToLong( + QueueUtilities.getPathToSequenceNumberFunction() + ).max().orElse(0); + + long nextSequenceNumber = sequenceNumber + 1; + + // create next + return new File( + queueDirectory.toAbsolutePath() + + File.separator + + queueNamePrefix + + "." + + nextSequenceNumber + ); + } + catch (UncheckedIOException uncheckedIOException) { + // just retry, reader modified the directory + return getNextWritableFilename(); + } + } + + public File getNextWritableFile() throws IOException { + return getNextWritableFilename(); + } +} \ No newline at end of file From 5da424778a19e126d22eceb0c7243578b0faf239 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Thu, 14 Dec 2023 16:14:06 +0200 Subject: [PATCH 006/146] Implementing AVRO serialization to Kafka record processing, WIP2. --- src/main/avro/KafkaRecord.avsc | 10 +- .../consumers/kafka/DatabaseOutput.java | 157 +++++++++++++++++- 2 files changed, 158 insertions(+), 9 deletions(-) diff --git a/src/main/avro/KafkaRecord.avsc b/src/main/avro/KafkaRecord.avsc index 483ced9e..0004dfcb 100644 --- a/src/main/avro/KafkaRecord.avsc +++ b/src/main/avro/KafkaRecord.avsc @@ -2,6 +2,14 @@ "type": "record", "name": "SyslogRecord", "fields": [ - {"name": "content", "type": "bytes"} + {"name": "timestamp", "type": "long"}, + {"name": "message", "type": "string"}, + {"name": "directory", "type": "string"}, + {"name": "stream", "type": "string"}, + {"name": "host", "type": "string"}, + {"name": "input", "type": "string"}, + {"name": "partition", "type": "string"}, + {"name": "offset", "type": "long"}, + {"name": "origin", "type": "string"} ] } \ No newline at end of file diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java index a2bc9216..9ca15c9e 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java @@ -13,6 +13,7 @@ import java.io.*; import java.time.Instant; +import java.time.ZonedDateTime; import java.util.List; import java.util.function.Consumer; @@ -46,6 +47,16 @@ public class DatabaseOutput implements Consumer> { private final long minimumFreeSpace; private final long maximumFileSize; private final WritableQueue writableQueue; // TODO: Implement the WritableQueue for managing the stream of records. + private final ByteBuffer sourceConcatenationBuffer; + private final SDVector teragrepStreamName; + private final SDVector teragrepDirectory; + private final SDVector eventNodeSourceSource; + private final SDVector eventNodeRelaySource; + private final SDVector eventNodeSourceSourceModule; + private final SDVector eventNodeRelaySourceModule; + private final SDVector eventNodeSourceHostname; + private final SDVector eventNodeRelayHostname; + private final SDVector originHostname; DatabaseOutput( Config config, @@ -56,8 +67,8 @@ public class DatabaseOutput implements Consumer> { this.table = table; this.runtimeStatistics = runtimeStatistics; this.topicCounter = topicCounter; - this.minimumFreeSpace = 32; // TODO: CHECK RIGHT VALUE FOR minimumFreeSpace - this.maximumFileSize = 64; + this.minimumFreeSpace = 32000000; // TODO: CHECK RIGHT VALUE FOR minimumFreeSpace + this.maximumFileSize = 64000000; // TODO: CHECK RIGHT VALUE FOR maximumFileSize. Maximum file size should be 64M. // TODO: Extract queueDirectory and queueNamePrefix values from config! // queueDirectory (folder) is named based on topic name, something like this: hdfs:///opt/teragrep/cfe_39/srv/topic_name // queueNamePrefix is partition, for example 0 @@ -66,13 +77,25 @@ public class DatabaseOutput implements Consumer> { "queueDirectory", "queueNamePrefix" ); + this.sourceConcatenationBuffer = ByteBuffer.allocateDirect(256*1024); + teragrepStreamName = new SDVector("teragrep@48577","streamname"); + teragrepDirectory = new SDVector("teragrep@48577","directory"); + this.eventNodeSourceSource = new SDVector("event_node_source@48577","source"); + this.eventNodeRelaySource = new SDVector("event_node_relay@48577","source"); + this.eventNodeSourceSourceModule = new SDVector("event_node_source@48577","source_module"); + this.eventNodeRelaySourceModule = new SDVector("event_node_relay@48577","source_module"); + this.eventNodeSourceHostname = new SDVector("event_node_source@48577","hostname"); + this.eventNodeRelayHostname = new SDVector("event_node_relay@48577","hostname"); + this.originHostname = new SDVector("origin@48577","hostname"); } boolean checkSizeTooLarge(long fileSize) { try { // If the syslogAvroWriter is already initialized, check the filesize so it doesn't go above 64M. if (fileSize > maximumFileSize) { - // file too large + // file too large for adding the new record, write the still adequately sized AVRO-file to the HDFS database and create a new empty AVRO-file. + + HDFSWriter.commit(); // TODO: Implement module for writing the adequately sized AVRO-file to the HDFS database. Needs to have the topic, partition and offset values of the last Kafka-record that was written to the AVRO-file as input parameters. syslogAvroWriter.close(); File syslogFile = @@ -86,6 +109,16 @@ boolean checkSizeTooLarge(long fileSize) { return false; } + private long rfc3339ToEpoch(ZonedDateTime zonedDateTime) { + final Instant instant = zonedDateTime.toInstant(); + + final long MICROS_PER_SECOND = 1000L * 1000L; + final long NANOS_PER_MICROS = 1000L; + final long sec = Math.multiplyExact(instant.getEpochSecond(), MICROS_PER_SECOND); + + return Math.addExact(sec, instant.getNano() / NANOS_PER_MICROS); + } + @Override public void accept(List recordOffsetObjectList) { long thisTime = Instant.now().toEpochMilli(); @@ -135,11 +168,42 @@ public void accept(List recordOffsetObjectList) { // Write all the data into a file using AVRO. // The size of each AVRO-serialized file should be as close to 64M as possible, and the name of the file should be set based on topic+partition+offset. - // TODO: Include stream (aka sourcetype) and directory (aka index) as well in syslogRecord. It only has record at the moment. Basically make the schema similar to that was used in mariadb... - syslogRecord.setContent(ByteBuffer.wrap(byteArray)); // byteArray is the loop's current recordOffsetObject.record + /* + EXAMPLE FORMAT FROM PTH_06.KafkaReconrdConverter WHICH SHOULD BE USED FOR AVRO-FORMAT! + return RowFactory.create( + Timestamp.from(instant), // 0 "_time", DataTypes.TimestampType + UTF8String.fromBytes(message).toString(), // 1 "_raw", DataTypes.StringType + UTF8String.fromBytes(index).toString(), // 2 "directory", DataTypes.StringType + UTF8String.fromBytes(sourcetype).toString(),// 3 "stream", DataTypes.StringType + UTF8String.fromBytes(hostname).toString(), // 4 "host", DataTypes.StringType, + UTF8String.fromBytes(input).toString(), // 5 "input", DataTypes.StringType + partition, // 6 "partition", DataTypes.StringType + offset, // 7 "offset", DataTypes.LongType + UTF8String.fromBytes(origin).toString() // 8 "origin", DataTypes.StringType + ); + */ + + // input + final byte[] source = eventToSource(); + + // origin + final byte[] origin = eventToOrigin(); + + // TODO: Format: Use AVRO format with syslog columns as indexed ones + final long epochMicros = rfc3339ToEpoch(new RFC5424Timestamp(rfc5424Frame.timestamp).toZonedDateTime()); + syslogRecord.setTimestamp(epochMicros); + syslogRecord.setMessage(rfc5424Frame.msg.toString()); + syslogRecord.setDirectory(rfc5424Frame.structuredData.getValue(teragrepDirectory).toString()); + syslogRecord.setStream(rfc5424Frame.structuredData.getValue(teragrepStreamName).toString()); + syslogRecord.setHost(rfc5424Frame.hostname.toString()); + syslogRecord.setInput(source.toString()); + syslogRecord.setPartition(recordOffsetObject.partition.toString()); + syslogRecord.setOffset(recordOffsetObject.offset); + syslogRecord.setOrigin(origin.toString()); + // Calculate the size of syslogRecord that is going to be written to syslogAvroWriter-file. long capacity = syslogRecord.toByteBuffer().capacity(); - checkSizeTooLarge(syslogAvroWriter.getFileSize() + capacity); // TODO: Fix the checkSizeTooLarge functionality so the closed syslogAvroWriter-file can be transferred to HDFS. + checkSizeTooLarge(syslogAvroWriter.getFileSize() + capacity); syslogAvroWriter.write(syslogRecord); /*new RFC5424Timestamp(rfc5424Frame.timestamp).toZonedDateTime().toInstant().getEpochSecond(); @@ -152,10 +216,10 @@ public void accept(List recordOffsetObjectList) { } } - // Handle possible "leftover" syslogRecords from the loop. TODO: Most likely borked like this, fix it in testing. + // Handle possible "leftover" syslogRecords from the loop. TODO: Most likely borked like this because AVRO-file should always have the schema stored in it. Maybe set syslogAvroWriter to null after every HDFSWriter.commit() etc. try { if (syslogAvroWriter.getFileSize() > 0) { - syslogAvroWriter.write(syslogRecord); + HDFSWriter.commit(); // commits the final AVRO-file to HDFS. } } catch (IOException e) { throw new RuntimeException(e); @@ -245,4 +309,81 @@ public void accept(List recordOffsetObjectList) { ); lastTimeCalled = Instant.now().toEpochMilli(); } + + + + private byte[] eventToOrigin() { + byte[] origin; + Fragment originFragment = rfc5424Frame.structuredData.getValue(originHostname); + if (!originFragment.isStub) { + origin = originFragment.toBytes(); + } + else { + origin = new byte[]{}; + } + return origin; + } + + private byte[] eventToSource() { + //input is produced from SD element event_node_source@48577 by + // concatenating "source_module:hostname:source". in case + //if event_node_source@48577 is not available use event_node_relay@48577. + //If neither are present, use null value. + + sourceConcatenationBuffer.clear(); + + Fragment sourceModuleFragment = rfc5424Frame.structuredData.getValue(eventNodeSourceSourceModule); + if (sourceModuleFragment.isStub) { + sourceModuleFragment = rfc5424Frame.structuredData.getValue(eventNodeRelaySourceModule); + } + + byte[] source_module; + if (!sourceModuleFragment.isStub) { + source_module = sourceModuleFragment.toBytes(); + } + else { + source_module = new byte[]{}; + } + + Fragment sourceHostnameFragment = rfc5424Frame.structuredData.getValue(eventNodeSourceHostname); + if (sourceHostnameFragment.isStub) { + sourceHostnameFragment = rfc5424Frame.structuredData.getValue(eventNodeRelayHostname); + } + + byte[] source_hostname; + if (!sourceHostnameFragment.isStub) { + source_hostname = sourceHostnameFragment.toBytes(); + } + else { + source_hostname = new byte[]{}; + } + + + Fragment sourceSourceFragment = rfc5424Frame.structuredData.getValue(eventNodeSourceSource); + if (sourceHostnameFragment.isStub) { + sourceSourceFragment = rfc5424Frame.structuredData.getValue(eventNodeRelaySource); + } + + byte[] source_source; + if (!sourceSourceFragment.isStub) { + source_source = sourceSourceFragment.toBytes(); + } + else { + source_source = new byte[]{}; + } + + + // source_module:hostname:source" + sourceConcatenationBuffer.put(source_module); + sourceConcatenationBuffer.put((byte) ':'); + sourceConcatenationBuffer.put(source_hostname); + sourceConcatenationBuffer.put((byte)':'); + sourceConcatenationBuffer.put(source_source); + + sourceConcatenationBuffer.flip(); + byte[] input = new byte[sourceConcatenationBuffer.remaining()]; + sourceConcatenationBuffer.get(input); + + return input; + } } \ No newline at end of file From 200d958d871c736f54b2ef06e4b8d42bd106d3d4 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Fri, 15 Dec 2023 15:57:22 +0200 Subject: [PATCH 007/146] Implementing HDFSWriter to Kafka consumer record processing. --- .../consumers/kafka/DatabaseOutput.java | 15 ++-- .../cfe_39/consumers/kafka/HDFSWriter.java | 79 +++++++++++++++++++ 2 files changed, 88 insertions(+), 6 deletions(-) create mode 100644 src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java index 9ca15c9e..75ccb151 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java @@ -119,6 +119,7 @@ private long rfc3339ToEpoch(ZonedDateTime zonedDateTime) { return Math.addExact(sec, instant.getNano() / NANOS_PER_MICROS); } + // Input parameter is a list of RecordOffsetObjects. Each object contains a record and its metadata (topic, partition and offset). @Override public void accept(List recordOffsetObjectList) { long thisTime = Instant.now().toEpochMilli(); @@ -194,7 +195,7 @@ public void accept(List recordOffsetObjectList) { syslogRecord.setTimestamp(epochMicros); syslogRecord.setMessage(rfc5424Frame.msg.toString()); syslogRecord.setDirectory(rfc5424Frame.structuredData.getValue(teragrepDirectory).toString()); - syslogRecord.setStream(rfc5424Frame.structuredData.getValue(teragrepStreamName).toString()); + syslogRecord.setStream(rfc5424Frame.structuredData.getValue(teragrepStreamName).toString()); // Or is sourcetype/stream supposed to be rfc5424Frame.appName.toString() instead? syslogRecord.setHost(rfc5424Frame.hostname.toString()); syslogRecord.setInput(source.toString()); syslogRecord.setPartition(recordOffsetObject.partition.toString()); @@ -203,7 +204,9 @@ public void accept(List recordOffsetObjectList) { // Calculate the size of syslogRecord that is going to be written to syslogAvroWriter-file. long capacity = syslogRecord.toByteBuffer().capacity(); + // Check if there is still room in syslogAvroWriter for another syslogRecord. Commit syslogAvroWriter to HDFS if no room left, emptying it out in the process. checkSizeTooLarge(syslogAvroWriter.getFileSize() + capacity); + // Add syslogRecord to syslogAvroWriter which has rooom for new syslogRecord. syslogAvroWriter.write(syslogRecord); /*new RFC5424Timestamp(rfc5424Frame.timestamp).toZonedDateTime().toInstant().getEpochSecond(); @@ -216,20 +219,20 @@ public void accept(List recordOffsetObjectList) { } } - // Handle possible "leftover" syslogRecords from the loop. TODO: Most likely borked like this because AVRO-file should always have the schema stored in it. Maybe set syslogAvroWriter to null after every HDFSWriter.commit() etc. + // Handle the "leftover" syslogRecords from the loop. try { - if (syslogAvroWriter.getFileSize() > 0) { + if (syslogAvroWriter != null) { HDFSWriter.commit(); // commits the final AVRO-file to HDFS. } } catch (IOException e) { throw new RuntimeException(e); } - // TODO: BELOW STUFF IS GOING TO BE EITHER SCRAPPED OR MOVED TO ANOTHER METHOD WHICH IS THEN CALLED SEPARATELY! EVERYTHING SHOULD BE DONE WITHIN THE ABOVE LOOP! + // TODO: BELOW STUFF IS GOING TO BE EITHER SCRAPPED OR MOVED TO HDFSWriter.java METHODS WHICH ARE THEN CALLED SEPARATELY! EVERYTHING SHOULD BE DONE WITHIN THE ABOVE LOOP! long start = Instant.now().toEpochMilli(); - // Add the code for sending the AVRO-serialized data to HDFS here, performance is measured between the start/end. + /*// Add the code for sending the AVRO-serialized data to HDFS here, performance is measured between the start/end. // Also remember to implement Kerberized access to HDFS. String hdfsuri = ""; // Get from config. @@ -273,7 +276,7 @@ public void accept(List recordOffsetObjectList) { } catch (IOException e) { throw new RuntimeException(e); - } + }*/ // TODO END diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java new file mode 100644 index 00000000..18c23b1b --- /dev/null +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java @@ -0,0 +1,79 @@ +package com.teragrep.cfe_39.consumers.kafka; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocalFileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.DistributedFileSystem; + +import java.io.IOException; +import java.net.URI; + +public class HDFSWriter implements AutoCloseable{ + + // TODO: Add input parameters + public HDFSWriter() { + // TODO: Code for initializing the class + // Add the code for sending the AVRO-serialized data to HDFS here, performance is measured between the start/end. + // Also remember to implement Kerberized access to HDFS. + String hdfsuri = ""; // Get from config. + + String path="/user/hdfs/example/hdfs/"; + String fileName= recordOffsetObjectList+".csv"; // FIXME + String fileContent="hello;world"; + + // ====== Init HDFS File System Object + Configuration conf = new Configuration(); + // Set FileSystem URI + conf.set("fs.defaultFS", hdfsuri); + // Because of Maven + conf.set("fs.hdfs.impl", DistributedFileSystem.class.getName()); + conf.set("fs.file.impl", LocalFileSystem.class.getName()); + // Set HADOOP user + System.setProperty("HADOOP_USER_NAME", "hdfs"); + System.setProperty("hadoop.home.dir", "/"); + } + + // TODO: Add input parameters + public void commit() { + // TODO: code for committing the AVRO-file to HDFS + //Get the filesystem - HDFS + try { + FileSystem fs = FileSystem.get(URI.create(hdfsuri), conf); + + //==== Create folder if not exists + Path workingDir=fs.getWorkingDirectory(); + Path newFolderPath= new Path(path); + if(!fs.exists(newFolderPath)) { + // Create new Directory + fs.mkdirs(newFolderPath); + // logger.info("Path "+path+" created."); + } + + //==== Write file + // logger.info("Begin Write file into hdfs"); + //Create a path + Path hdfswritepath = new Path(newFolderPath + "/" + fileName); + //Init output stream + FSDataOutputStream outputStream=fs.create(hdfswritepath); + //Cassical output stream usage + outputStream.writeBytes(fileContent); + outputStream.close(); + // logger.info("End Write file into hdfs"); + + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + public void add() { + // TODO: code for adding file for commit + } + + public void close() { + + } + + +} From 281d86caf3fd39cadc2e839bd8369ae49c8cc7c5 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Mon, 18 Dec 2023 16:38:38 +0200 Subject: [PATCH 008/146] Implementing HDFSWriter to Kafka consumer record processing (2). --- .../consumers/kafka/DatabaseOutput.java | 22 ++++++++++---- .../cfe_39/consumers/kafka/HDFSWriter.java | 29 ++++++++++--------- .../consumers/kafka/SyslogAvroWriter.java | 2 +- 3 files changed, 34 insertions(+), 19 deletions(-) diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java index 75ccb151..2e9560a4 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java @@ -57,6 +57,7 @@ public class DatabaseOutput implements Consumer> { private final SDVector eventNodeSourceHostname; private final SDVector eventNodeRelayHostname; private final SDVector originHostname; + private File syslogFile; DatabaseOutput( Config config, @@ -95,9 +96,14 @@ boolean checkSizeTooLarge(long fileSize) { if (fileSize > maximumFileSize) { // file too large for adding the new record, write the still adequately sized AVRO-file to the HDFS database and create a new empty AVRO-file. - HDFSWriter.commit(); // TODO: Implement module for writing the adequately sized AVRO-file to the HDFS database. Needs to have the topic, partition and offset values of the last Kafka-record that was written to the AVRO-file as input parameters. - + // This part closes the writing of now "complete" AVRO-file and stores the file to HDFS. syslogAvroWriter.close(); + try (HDFSWriter writer = new HDFSWriter()) { + writer.commit(syslogFile); // TODO: Implement module for writing the adequately sized AVRO-file to the HDFS database. Needs to have the topic, partition and offset values of the last Kafka-record that was written to the AVRO-file as input parameters. + } + // TODO: Delete AVRO-files that have been committed to HDFS? + + // This part defines a new empty file to which the new AVRO-serialized records are stored until it again hits the 64M size limit. File syslogFile = writableQueue.getNextWritableFile(); syslogAvroWriter = new SyslogAvroWriter(syslogFile); @@ -140,7 +146,7 @@ public void accept(List recordOffsetObjectList) { // Initializing syslogAvroWriter. if (syslogAvroWriter == null) { try { - File syslogFile = + syslogFile = writableQueue.getNextWritableFile(); // TODO: Check how topic name, partition and offset should be added to the HDFS filename. // The avro serialization filename shouldn't really matter as long as the name is changed when stuff is stored to HDFS. @@ -222,7 +228,11 @@ public void accept(List recordOffsetObjectList) { // Handle the "leftover" syslogRecords from the loop. try { if (syslogAvroWriter != null) { - HDFSWriter.commit(); // commits the final AVRO-file to HDFS. + syslogAvroWriter.close(); + try (HDFSWriter writer = new HDFSWriter()) { + writer.commit(syslogFile); // commits the final AVRO-file to HDFS. + } + // TODO: Delete AVRO-files that have been committed to HDFS? } } catch (IOException e) { throw new RuntimeException(e); @@ -230,7 +240,7 @@ public void accept(List recordOffsetObjectList) { // TODO: BELOW STUFF IS GOING TO BE EITHER SCRAPPED OR MOVED TO HDFSWriter.java METHODS WHICH ARE THEN CALLED SEPARATELY! EVERYTHING SHOULD BE DONE WITHIN THE ABOVE LOOP! - long start = Instant.now().toEpochMilli(); + /*// Add the code for sending the AVRO-serialized data to HDFS here, performance is measured between the start/end. // Also remember to implement Kerberized access to HDFS. @@ -280,6 +290,8 @@ public void accept(List recordOffsetObjectList) { // TODO END + long start = Instant.now().toEpochMilli(); + // Measures performance of code that is between start and end. long end = Instant.now().toEpochMilli(); long took = (end - start); diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java index 18c23b1b..3ebed26d 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java @@ -7,24 +7,30 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.DistributedFileSystem; +import java.io.File; import java.io.IOException; import java.net.URI; public class HDFSWriter implements AutoCloseable{ - // TODO: Add input parameters + private String fileName; + private String fileContent; + private String path; + private String hdfsuri; + private Configuration conf; + + // TODO: Add input parameters, for example config. public HDFSWriter() { // TODO: Code for initializing the class - // Add the code for sending the AVRO-serialized data to HDFS here, performance is measured between the start/end. // Also remember to implement Kerberized access to HDFS. - String hdfsuri = ""; // Get from config. + hdfsuri = ""; // Get from config. - String path="/user/hdfs/example/hdfs/"; - String fileName= recordOffsetObjectList+".csv"; // FIXME - String fileContent="hello;world"; + path="/user/hdfs/example/hdfs/"; + fileName= "test.csv"; // FIXME + fileContent="hello;world"; // ====== Init HDFS File System Object - Configuration conf = new Configuration(); + conf = new Configuration(); // Set FileSystem URI conf.set("fs.defaultFS", hdfsuri); // Because of Maven @@ -35,9 +41,10 @@ public HDFSWriter() { System.setProperty("hadoop.home.dir", "/"); } - // TODO: Add input parameters - public void commit() { + // TODO: Add more input parameters that are needed for creating the proper filename for HDFS. At least Topic, Partition and Offset. + public void commit(File syslogFile) { // TODO: code for committing the AVRO-file to HDFS + //Get the filesystem - HDFS try { FileSystem fs = FileSystem.get(URI.create(hdfsuri), conf); @@ -67,10 +74,6 @@ public void commit() { } } - public void add() { - // TODO: code for adding file for commit - } - public void close() { } diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java index 1c6f29ac..99be99fa 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java @@ -46,7 +46,7 @@ class SyslogAvroWriter implements AutoCloseable { void write(SyslogRecord syslogRecord) throws IOException{ dataFileWriter.append(syslogRecord); - dataFileWriter.flush(); // TODO: Avro files 'flush' must be called as few times as possible. Check memory usage impact + // dataFileWriter.flush(); // TODO: Avro files 'flush' must be called as few times as possible. Check memory usage impact. Use only automatic flush which is triggered when .close() is called? } public void close() throws IOException { From 6f4b9efc8c752148bdc820cffec3432094035981 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Tue, 19 Dec 2023 15:31:02 +0200 Subject: [PATCH 009/146] Implementing HDFSWriter to Kafka consumer record processing (3). --- src/main/java/com/teragrep/cfe_39/Config.java | 27 ++++ .../consumers/kafka/DatabaseOutput.java | 115 +++++------------- .../cfe_39/consumers/kafka/HDFSWriter.java | 58 +++++---- 3 files changed, 93 insertions(+), 107 deletions(-) diff --git a/src/main/java/com/teragrep/cfe_39/Config.java b/src/main/java/com/teragrep/cfe_39/Config.java index 17816d11..35edb06f 100644 --- a/src/main/java/com/teragrep/cfe_39/Config.java +++ b/src/main/java/com/teragrep/cfe_39/Config.java @@ -33,6 +33,10 @@ public class Config { private final int createPartitionsInAdvanceHours; private final boolean overrideTableLocation; private final String tableLocation; + private final String hdfsPath; + private final String hdfsuri; + private final String queueDirectory; + private final String queueNamePrefix; Config() throws IOException { Properties properties = new Properties(); @@ -77,6 +81,15 @@ public class Config { throw new IllegalArgumentException("db.tableLocation resulted in empty string when db.overrideTableLocation was true"); } + // HDFS + this.hdfsPath = properties.getProperty("hdfsPath", "hdfs:///opt/teragrep/cfe_39/srv/"); + this.hdfsuri = properties.getProperty("hdfsuri", ""); + + // AVRO + this.queueDirectory = properties.getProperty("queueDirectory", ""); + this.queueNamePrefix = properties.getProperty("queueNamePrefix", ""); + + // kafka this.queueTopicPattern = properties.getProperty("queueTopicPattern", "^.*$"); @@ -131,6 +144,20 @@ public String getDbPassword() { return dbPassword; } + public String getHdfsPath() { + return hdfsPath; + } + public String getHdfsuri() { + return hdfsuri; + } + + public String getQueueDirectory() { + return queueDirectory; + } + public String getQueueNamePrefix() { + return queueNamePrefix; + } + public boolean isReplicationEnabled() { return replicationEnabled; } diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java index 2e9560a4..d7284bce 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java @@ -22,10 +22,8 @@ import java.net.URI; -// TODO: // The kafka stream should first be deserialized using rlo_06 and then serialized again using avro and stored in HDFS. // The target where the record is stored in HDFS is based on the topic, partition and offset. ie. topic_name/0.123456 where offset is 123456 -// First implement the AVRO serialization (saves the data into a file) and then implement the HDFS access for storing the data. public class DatabaseOutput implements Consumer> { private static final Logger LOGGER = LoggerFactory.getLogger(DatabaseOutput.class); @@ -42,11 +40,10 @@ public class DatabaseOutput implements Consumer> { public static final String ANSI_RESET = "\u001B[0m"; public static final String ANSI_GREEN = "\u001B[32m"; public static final String ANSI_BLUE = "\u001B[34m"; - private final SyslogRecord syslogRecord = new SyslogRecord(); private SyslogAvroWriter syslogAvroWriter; private final long minimumFreeSpace; private final long maximumFileSize; - private final WritableQueue writableQueue; // TODO: Implement the WritableQueue for managing the stream of records. + private final WritableQueue writableQueue; private final ByteBuffer sourceConcatenationBuffer; private final SDVector teragrepStreamName; private final SDVector teragrepDirectory; @@ -58,6 +55,7 @@ public class DatabaseOutput implements Consumer> { private final SDVector eventNodeRelayHostname; private final SDVector originHostname; private File syslogFile; + private final Config config; DatabaseOutput( Config config, @@ -65,19 +63,19 @@ public class DatabaseOutput implements Consumer> { RuntimeStatistics runtimeStatistics, TopicCounter topicCounter ) { + this.config = config; this.table = table; this.runtimeStatistics = runtimeStatistics; this.topicCounter = topicCounter; this.minimumFreeSpace = 32000000; // TODO: CHECK RIGHT VALUE FOR minimumFreeSpace this.maximumFileSize = 64000000; // TODO: CHECK RIGHT VALUE FOR maximumFileSize. Maximum file size should be 64M. - // TODO: Extract queueDirectory and queueNamePrefix values from config! - // queueDirectory (folder) is named based on topic name, something like this: hdfs:///opt/teragrep/cfe_39/srv/topic_name - // queueNamePrefix is partition, for example 0 - // The end result that is produced in WritableQueue is hdfs:///opt/teragrep/cfe_39/srv/topic_name/0.12345 where 12345 is offset and 0 the partition. + + // queueDirectory and queueNamePrefix shouldn't be critical to name according to the HDFS requirements (topic+partition+offset for filename) as it's just used for storing the AVRO-serialized files. this.writableQueue = new WritableQueue( - "queueDirectory", - "queueNamePrefix" + config.getQueueDirectory(), + config.getQueueNamePrefix() ); + this.sourceConcatenationBuffer = ByteBuffer.allocateDirect(256*1024); teragrepStreamName = new SDVector("teragrep@48577","streamname"); teragrepDirectory = new SDVector("teragrep@48577","directory"); @@ -90,7 +88,7 @@ public class DatabaseOutput implements Consumer> { this.originHostname = new SDVector("origin@48577","hostname"); } - boolean checkSizeTooLarge(long fileSize) { + boolean checkSizeTooLarge(long fileSize, RecordOffsetObject lastObject) { try { // If the syslogAvroWriter is already initialized, check the filesize so it doesn't go above 64M. if (fileSize > maximumFileSize) { @@ -98,8 +96,8 @@ boolean checkSizeTooLarge(long fileSize) { // This part closes the writing of now "complete" AVRO-file and stores the file to HDFS. syslogAvroWriter.close(); - try (HDFSWriter writer = new HDFSWriter()) { - writer.commit(syslogFile); // TODO: Implement module for writing the adequately sized AVRO-file to the HDFS database. Needs to have the topic, partition and offset values of the last Kafka-record that was written to the AVRO-file as input parameters. + try (HDFSWriter writer = new HDFSWriter(config, lastObject)) { + writer.commit(syslogFile); // commits the final AVRO-file to HDFS. } // TODO: Delete AVRO-files that have been committed to HDFS? @@ -134,21 +132,22 @@ public void accept(List recordOffsetObjectList) { LOGGER.debug(ANSI_BLUE + "Fuura searching your batch for <[" + table + "]> with records <" + recordOffsetObjectList.size() + "> and took <" + (ftook) + "> milliseconds. <" + (recordOffsetObjectList.size() * 1000L / ftook) + "> EPS. " + ANSI_RESET); long batchBytes = 0L; - // TODO: The recordOffsetObjectList loop will go through all the objects in the list. + // The recordOffsetObjectList loop will go through all the objects in the list. // While it goes through the list, the contents of the objects are serialized into an AVRO-file. // When the file size is about to go above 64M, commit the file into HDFS using the latest topic/partition/offset values as the filename and start fresh with an empty AVRO-file. // Serialize the object that was going to make the file go above 64M into the now empty AVRO-file and continue the loop. // https://pagure.xnet.fi/com-teragrep/rlo_09/blob/avroness/f/src/main/java/com/teragrep/rlo_09/SyslogAvroWriter.java // https://pagure.xnet.fi/com-teragrep/rlo_09/blob/avroness/f/src/main/java/com/teragrep/rlo_09/WriteCoordinator.java // every recordOffsetObject.record on the recordOffsetObjectList basically represents a rlo_09 WriteCoordinator.accept(byte[] bytes) when the list is gone through in a loop. + RecordOffsetObject lastObject = null; + long start = Instant.now().toEpochMilli(); // Starts measuring performance here. Measures how long it takes to process the whole recordOffsetObjectList. for (RecordOffsetObject recordOffsetObject : recordOffsetObjectList) { - + lastObject = recordOffsetObject; // Initializing syslogAvroWriter. if (syslogAvroWriter == null) { try { syslogFile = writableQueue.getNextWritableFile(); - // TODO: Check how topic name, partition and offset should be added to the HDFS filename. // The avro serialization filename shouldn't really matter as long as the name is changed when stuff is stored to HDFS. // And the content of the AVRO-serialized file that is going to be stored in HDFS is finalized only when the maximumFileSize has been reached. // This means the HDFS filename is only finalized when the AVRO-serialized file is finalized, because every Kafka-record added to the file is going to change the offset that is going to be used for the filename. @@ -158,7 +157,7 @@ public void accept(List recordOffsetObjectList) { } } else { try { - checkSizeTooLarge(syslogAvroWriter.getFileSize()); + checkSizeTooLarge(syslogAvroWriter.getFileSize(), lastObject); } catch (IOException ioException) { throw new UncheckedIOException(ioException); } @@ -173,7 +172,7 @@ public void accept(List recordOffsetObjectList) { // rfc5424Frame has loaded the record data, it's ready for deserialization. // Implement AVRO serialization for the Kafka records here, preparing the data for writing to HDFS. // Write all the data into a file using AVRO. - // The size of each AVRO-serialized file should be as close to 64M as possible, and the name of the file should be set based on topic+partition+offset. + // The size of each AVRO-serialized file should be as close to 64M as possible. /* EXAMPLE FORMAT FROM PTH_06.KafkaReconrdConverter WHICH SHOULD BE USED FOR AVRO-FORMAT! @@ -196,22 +195,25 @@ public void accept(List recordOffsetObjectList) { // origin final byte[] origin = eventToOrigin(); - // TODO: Format: Use AVRO format with syslog columns as indexed ones + // Format: Use AVRO format with syslog columns as indexed ones final long epochMicros = rfc3339ToEpoch(new RFC5424Timestamp(rfc5424Frame.timestamp).toZonedDateTime()); - syslogRecord.setTimestamp(epochMicros); - syslogRecord.setMessage(rfc5424Frame.msg.toString()); - syslogRecord.setDirectory(rfc5424Frame.structuredData.getValue(teragrepDirectory).toString()); - syslogRecord.setStream(rfc5424Frame.structuredData.getValue(teragrepStreamName).toString()); // Or is sourcetype/stream supposed to be rfc5424Frame.appName.toString() instead? - syslogRecord.setHost(rfc5424Frame.hostname.toString()); - syslogRecord.setInput(source.toString()); - syslogRecord.setPartition(recordOffsetObject.partition.toString()); - syslogRecord.setOffset(recordOffsetObject.offset); - syslogRecord.setOrigin(origin.toString()); + SyslogRecord syslogRecord = SyslogRecord.newBuilder() + .setTimestamp(epochMicros) + .setMessage(rfc5424Frame.msg.toString()) + .setDirectory(rfc5424Frame.structuredData.getValue(teragrepDirectory).toString()) + .setStream(rfc5424Frame.structuredData.getValue(teragrepStreamName).toString()) // Or is sourcetype/stream supposed to be rfc5424Frame.appName.toString() instead? + .setHost(rfc5424Frame.hostname.toString()) + .setInput(source.toString()) + .setPartition(recordOffsetObject.partition.toString()) + .setOffset(recordOffsetObject.offset) + .setOrigin(origin.toString()) + .build(); + // Calculate the size of syslogRecord that is going to be written to syslogAvroWriter-file. long capacity = syslogRecord.toByteBuffer().capacity(); // Check if there is still room in syslogAvroWriter for another syslogRecord. Commit syslogAvroWriter to HDFS if no room left, emptying it out in the process. - checkSizeTooLarge(syslogAvroWriter.getFileSize() + capacity); + checkSizeTooLarge(syslogAvroWriter.getFileSize() + capacity, lastObject); // Add syslogRecord to syslogAvroWriter which has rooom for new syslogRecord. syslogAvroWriter.write(syslogRecord); @@ -229,7 +231,7 @@ public void accept(List recordOffsetObjectList) { try { if (syslogAvroWriter != null) { syslogAvroWriter.close(); - try (HDFSWriter writer = new HDFSWriter()) { + try (HDFSWriter writer = new HDFSWriter(config, lastObject)) { writer.commit(syslogFile); // commits the final AVRO-file to HDFS. } // TODO: Delete AVRO-files that have been committed to HDFS? @@ -238,59 +240,6 @@ public void accept(List recordOffsetObjectList) { throw new RuntimeException(e); } - // TODO: BELOW STUFF IS GOING TO BE EITHER SCRAPPED OR MOVED TO HDFSWriter.java METHODS WHICH ARE THEN CALLED SEPARATELY! EVERYTHING SHOULD BE DONE WITHIN THE ABOVE LOOP! - - - - /*// Add the code for sending the AVRO-serialized data to HDFS here, performance is measured between the start/end. - // Also remember to implement Kerberized access to HDFS. - String hdfsuri = ""; // Get from config. - - String path="/user/hdfs/example/hdfs/"; - String fileName= recordOffsetObjectList+".csv"; - String fileContent="hello;world"; - - // ====== Init HDFS File System Object - Configuration conf = new Configuration(); - // Set FileSystem URI - conf.set("fs.defaultFS", hdfsuri); - // Because of Maven - conf.set("fs.hdfs.impl", DistributedFileSystem.class.getName()); - conf.set("fs.file.impl", LocalFileSystem.class.getName()); - // Set HADOOP user - System.setProperty("HADOOP_USER_NAME", "hdfs"); - System.setProperty("hadoop.home.dir", "/"); - //Get the filesystem - HDFS - try { - FileSystem fs = FileSystem.get(URI.create(hdfsuri), conf); - - //==== Create folder if not exists - Path workingDir=fs.getWorkingDirectory(); - Path newFolderPath= new Path(path); - if(!fs.exists(newFolderPath)) { - // Create new Directory - fs.mkdirs(newFolderPath); - // logger.info("Path "+path+" created."); - } - - //==== Write file - // logger.info("Begin Write file into hdfs"); - //Create a path - Path hdfswritepath = new Path(newFolderPath + "/" + fileName); - //Init output stream - FSDataOutputStream outputStream=fs.create(hdfswritepath); - //Cassical output stream usage - outputStream.writeBytes(fileContent); - outputStream.close(); - // logger.info("End Write file into hdfs"); - - } catch (IOException e) { - throw new RuntimeException(e); - }*/ - - // TODO END - - long start = Instant.now().toEpochMilli(); // Measures performance of code that is between start and end. long end = Instant.now().toEpochMilli(); diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java index 3ebed26d..dd71c7db 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java @@ -1,5 +1,6 @@ package com.teragrep.cfe_39.consumers.kafka; +import com.teragrep.cfe_39.Config; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; @@ -10,47 +11,50 @@ import java.io.File; import java.io.IOException; import java.net.URI; +import java.nio.file.Files; +import java.nio.file.Paths; public class HDFSWriter implements AutoCloseable{ - private String fileName; - private String fileContent; - private String path; - private String hdfsuri; - private Configuration conf; + private final String fileName; + private final String path; + private final FileSystem fs; - // TODO: Add input parameters, for example config. - public HDFSWriter() { - // TODO: Code for initializing the class - // Also remember to implement Kerberized access to HDFS. - hdfsuri = ""; // Get from config. + public HDFSWriter(Config config, RecordOffsetObject lastObject) { + // Code for initializing the class + // Also remember to implement Kerberized access to HDFS. + String hdfsuri = config.getHdfsuri(); // Get from config. - path="/user/hdfs/example/hdfs/"; - fileName= "test.csv"; // FIXME - fileContent="hello;world"; + // The filepath should be something like hdfs:///opt/teragrep/cfe_39/srv/topic_name/0.12345 where 12345 is offset and 0 the partition. + // These values should be fetched from config and other input parameters (topic+partition+offset). + path = config.getHdfsPath()+"/"+lastObject.topic; + fileName = lastObject.partition+"."+lastObject.offset; // filename should be constructed from partition and offset. // ====== Init HDFS File System Object - conf = new Configuration(); + Configuration conf = new Configuration(); // Set FileSystem URI conf.set("fs.defaultFS", hdfsuri); // Because of Maven conf.set("fs.hdfs.impl", DistributedFileSystem.class.getName()); conf.set("fs.file.impl", LocalFileSystem.class.getName()); - // Set HADOOP user + // Set HADOOP user here, Kerberus parameters most likely needs to be added here too. System.setProperty("HADOOP_USER_NAME", "hdfs"); System.setProperty("hadoop.home.dir", "/"); + // filesystem for HDFS access is set here + try { + fs = FileSystem.get(URI.create(hdfsuri), conf); + } catch (IOException e) { + throw new RuntimeException(e); + } } - // TODO: Add more input parameters that are needed for creating the proper filename for HDFS. At least Topic, Partition and Offset. + // Method for committing the AVRO-file to HDFS public void commit(File syslogFile) { - // TODO: code for committing the AVRO-file to HDFS - //Get the filesystem - HDFS try { - FileSystem fs = FileSystem.get(URI.create(hdfsuri), conf); - //==== Create folder if not exists Path workingDir=fs.getWorkingDirectory(); + // Sets the directory where the data should be stored, if the directory doesn't exist then it's created. Path newFolderPath= new Path(path); if(!fs.exists(newFolderPath)) { // Create new Directory @@ -61,11 +65,13 @@ public void commit(File syslogFile) { //==== Write file // logger.info("Begin Write file into hdfs"); //Create a path - Path hdfswritepath = new Path(newFolderPath + "/" + fileName); + Path hdfswritepath = new Path(newFolderPath + "/" + fileName); // filename should be set according to the requirements: 0.12345 where 0 is Kafka partition and 12345 is Kafka offset. //Init output stream FSDataOutputStream outputStream=fs.create(hdfswritepath); - //Cassical output stream usage - outputStream.writeBytes(fileContent); + // Write the file contents of syslogFile to hdfswritepath in HDFS. + // file to bytes[] + byte[] bytes = Files.readAllBytes(Paths.get(syslogFile.getPath())); + outputStream.write(bytes); outputStream.close(); // logger.info("End Write file into hdfs"); @@ -75,7 +81,11 @@ public void commit(File syslogFile) { } public void close() { - + try { + fs.close(); + } catch (IOException e) { + throw new RuntimeException(e); + } } From 00ce004735a083fc2e7f1d03316ad91f3eb22c3a Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Wed, 20 Dec 2023 16:49:58 +0200 Subject: [PATCH 010/146] Testing configurations, setting up testcases, debugging dependency issues. --- pom.xml | 24 ++++++++++++++ src/main/java/com/teragrep/cfe_39/Config.java | 12 +++---- .../consumers/kafka/DatabaseOutput.java | 4 +++ .../kafka/MockKafkaConsumerFactoryTemp.java | 1 - .../cfe_39/metrics/mxj/MXJBeanDynamizer.java | 8 +++-- .../cfe_39/metrics/topic/TopicStatistics.java | 19 +++++++---- .../teragrep/cfe_39/KafkaConsumerTest.java | 33 +++++++++++++++++++ 7 files changed, 84 insertions(+), 17 deletions(-) create mode 100644 src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java diff --git a/pom.xml b/pom.xml index 2137b210..225f8ad7 100644 --- a/pom.xml +++ b/pom.xml @@ -53,6 +53,30 @@ mxj_01 1.0.3 --> + + org.junit.jupiter + junit-jupiter-engine + 5.7.1 + test + + + org.junit.platform + junit-platform-launcher + 1.7.1 + test + + + org.junit.jupiter + junit-jupiter-api + 5.7.1 + test + + + org.junit.jupiter + junit-jupiter + 5.7.1 + test + com.teragrep rlo_06 diff --git a/src/main/java/com/teragrep/cfe_39/Config.java b/src/main/java/com/teragrep/cfe_39/Config.java index 35edb06f..4f81a496 100644 --- a/src/main/java/com/teragrep/cfe_39/Config.java +++ b/src/main/java/com/teragrep/cfe_39/Config.java @@ -15,10 +15,10 @@ public class Config { - // db +/* // db private final String dbConnectionUrl; private final String dbUsername; - private final String dbPassword; + private final String dbPassword;*/ // kafka private final String queueTopicPattern; @@ -45,7 +45,7 @@ public class Config { properties.load(Files.newInputStream(configPath)); LOGGER.debug("Got configuration: " + properties); - // db + /*// db this.dbConnectionUrl = properties.getProperty("db.connectionUrl"); if (this.dbConnectionUrl == null) { throw new IllegalArgumentException("db.connectionUrl not set"); @@ -57,7 +57,7 @@ public class Config { this.dbPassword = properties.getProperty("db.password"); if (this.dbPassword == null) { throw new IllegalArgumentException("db.password not set"); - } + }*/ String replicationEnabledString = properties.getProperty("db.replicationEnabled", "false"); this.replicationEnabled = Boolean.parseBoolean(replicationEnabledString); @@ -132,7 +132,7 @@ public String getTableLocation() { return tableLocation; } - public String getDbConnectionUrl() { +/* public String getDbConnectionUrl() { return dbConnectionUrl; } @@ -142,7 +142,7 @@ public String getDbUsername() { public String getDbPassword() { return dbPassword; - } + }*/ public String getHdfsPath() { return hdfsPath; diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java index d7284bce..bf77bbb2 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java @@ -25,6 +25,10 @@ // The kafka stream should first be deserialized using rlo_06 and then serialized again using avro and stored in HDFS. // The target where the record is stored in HDFS is based on the topic, partition and offset. ie. topic_name/0.123456 where offset is 123456 +// TODO: make testcases using MockKafkaConsumerFactory. The mock consumer is activated for testing using the configuration file: readerKafkaProperties.getProperty("useMockKafkaConsumer", "false") +// DONE: +// Thus the configuration file is the first thing to make for testcases, and the first testcases are for testing the configuration file. + public class DatabaseOutput implements Consumer> { private static final Logger LOGGER = LoggerFactory.getLogger(DatabaseOutput.class); private final RFC5424Frame rfc5424Frame = new RFC5424Frame(false); diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java index 9184b546..fb6df204 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java @@ -24,7 +24,6 @@ public class MockKafkaConsumerFactoryTemp { final static private Logger LOGGER = LoggerFactory.getLogger(MockKafkaConsumerFactoryTemp.class); private MockKafkaConsumerFactoryTemp() { - } private static void generateEvents(MockConsumer consumer) { diff --git a/src/main/java/com/teragrep/cfe_39/metrics/mxj/MXJBeanDynamizer.java b/src/main/java/com/teragrep/cfe_39/metrics/mxj/MXJBeanDynamizer.java index b7e9db89..b25027b7 100644 --- a/src/main/java/com/teragrep/cfe_39/metrics/mxj/MXJBeanDynamizer.java +++ b/src/main/java/com/teragrep/cfe_39/metrics/mxj/MXJBeanDynamizer.java @@ -1,6 +1,6 @@ package com.teragrep.cfe_39.metrics.mxj; -import com.teragrep.mxj_01.DynamicBean; // TODO: Fix the dependency failing to load. +// import com.teragrep.mxj_01.DynamicBean; // TODO: Fix the dependency failing to load. import javax.management.DynamicMBean; import java.util.List; @@ -13,7 +13,8 @@ public MXJBeanDynamizer(List mxjItems) { } public DynamicMBean createDynamicMBean() { - DynamicBean.Builder builder = DynamicBean.builder(); + // FIXME + /*DynamicBean.Builder builder = DynamicBean.builder(); for (MXJItem a : mxjItems) { builder = builder.withSimpleAttribute( @@ -24,6 +25,7 @@ public DynamicMBean createDynamicMBean() { ); } - return builder.build(); + return builder.build();*/ + return null; } } diff --git a/src/main/java/com/teragrep/cfe_39/metrics/topic/TopicStatistics.java b/src/main/java/com/teragrep/cfe_39/metrics/topic/TopicStatistics.java index 04ae7d76..49796684 100644 --- a/src/main/java/com/teragrep/cfe_39/metrics/topic/TopicStatistics.java +++ b/src/main/java/com/teragrep/cfe_39/metrics/topic/TopicStatistics.java @@ -1,9 +1,10 @@ package com.teragrep.cfe_39.metrics.topic; import com.teragrep.cfe_39.metrics.mxj.MXJEndpoint; -import com.teragrep.mxj_01.CompositeDataWriter; +// FIXME +/*import com.teragrep.mxj_01.CompositeDataWriter; import com.teragrep.mxj_01.DynamicBean; -import com.teragrep.mxj_01.TabularDataWriter; +import com.teragrep.mxj_01.TabularDataWriter; */ import javax.management.*; import java.util.List; @@ -11,12 +12,14 @@ public class TopicStatistics { private final List topicList; - private final DynamicBean dynamicBean; + // FIXME + // private final DynamicBean dynamicBean; public TopicStatistics(List topicList) { this.topicList = topicList; - // page + // FIXME +/* // page CompositeDataWriter topicCounterWriter = CompositeDataWriter.builder(TopicCounter.class) .withTypeName("topic") .withTypeDescription("Topic throughput") @@ -44,16 +47,18 @@ public TopicStatistics(List topicList) { () -> topicList, topicListWriter ) - .build(); + .build();*/ } public DynamicMBean register() { - MXJEndpoint mxjEndpoint = new MXJEndpoint( + // FIXME + /*MXJEndpoint mxjEndpoint = new MXJEndpoint( "com.teragrep.cfe_30", "Metrics", "Topic", dynamicBean ); - return mxjEndpoint.register(); + return mxjEndpoint.register();*/ + return null; } } diff --git a/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java b/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java new file mode 100644 index 00000000..f40890fe --- /dev/null +++ b/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java @@ -0,0 +1,33 @@ +package com.teragrep.cfe_39; + +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.util.Properties; + +public class KafkaConsumerTest { + + + @Test + public void kafkaConsumerTest() { + // TODO: make tests here + } + + @Test + public void configTest() { + // Configuration tests done, configurations working correctly with the right .jaas and .properties files. + try { + Config config = new Config(); + + Properties readerKafkaProperties = config.getKafkaConsumerProperties(); + + // Test extracting useMockKafkaConsumer value from config. + boolean useMockKafkaConsumer = Boolean.parseBoolean( + readerKafkaProperties.getProperty("useMockKafkaConsumer", "false") + ); + System.out.println("useMockKafkaConsumer: "+useMockKafkaConsumer); + } catch (IOException e) { + throw new RuntimeException(e); + } + } +} From 4e23a0659bbab2d47b5b01073ef2b81963932a97 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Fri, 22 Dec 2023 16:07:50 +0200 Subject: [PATCH 011/146] Testing and debugging kafka consumer processing, debugging dependency issues. --- .../consumers/kafka/KafkaController.java | 17 +++++++++++---- .../teragrep/cfe_39/KafkaConsumerTest.java | 21 ++++++++++++++++--- 2 files changed, 31 insertions(+), 7 deletions(-) diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java index a250f28b..58f5c60c 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java @@ -6,6 +6,7 @@ import com.teragrep.cfe_39.metrics.topic.TopicCounter; import com.teragrep.cfe_39.metrics.topic.TopicStatistics; import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.consumer.MockConsumer; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.slf4j.Logger; @@ -58,16 +59,24 @@ public class KafkaController { public KafkaController(Config config) { this.config = config; - this.kafkaConsumer = new KafkaConsumer<>(config.getKafkaConsumerProperties(), new ByteArrayDeserializer(), new ByteArrayDeserializer()); + Properties readerKafkaProperties = config.getKafkaConsumerProperties(); + boolean useMockKafkaConsumer = Boolean.parseBoolean( + readerKafkaProperties.getProperty("useMockKafkaConsumer", "false") + ); + if (useMockKafkaConsumer) { + this.kafkaConsumer = MockKafkaConsumerFactoryTemp.getConsumer(); + } else { + this.kafkaConsumer = new KafkaConsumer<>(config.getKafkaConsumerProperties(), new ByteArrayDeserializer(), new ByteArrayDeserializer()); + } } public void run() throws InterruptedException { // register runtime statistics - runtimeStatistics.register(); + // runtimeStatistics.register(); // FIXME // register duration statistics DurationStatistics durationStatistics = new DurationStatistics(); - durationStatistics.register(); + // durationStatistics.register(); // FIXME // register per topic counting List topicCounters = new CopyOnWriteArrayList<>(); @@ -120,7 +129,7 @@ private void createReader(String topic, List topicCounters) throws } private void topicScan(DurationStatistics durationStatistics, List topicCounters) { - Map> listTopics = kafkaConsumer.listTopics(Duration.ofSeconds(60)); + Map> listTopics = kafkaConsumer.listTopics(Duration.ofSeconds(60)); // TODO: The listTopics is empty, this means problems in mock kafka. Pattern topicsRegex = Pattern.compile(config.getQueueTopicPattern()); // Find the topics available in Kafka based on given QueueTopicPattern, both active and in-active. diff --git a/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java b/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java index f40890fe..496e2f75 100644 --- a/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java +++ b/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java @@ -1,5 +1,6 @@ package com.teragrep.cfe_39; +import com.teragrep.cfe_39.consumers.kafka.KafkaController; import org.junit.jupiter.api.Test; import java.io.IOException; @@ -7,10 +8,24 @@ public class KafkaConsumerTest { - + // TODO: make tests here. Make sure application.properties has consumer.useMockKafkaConsumer=true enabled for Kafka testing. @Test - public void kafkaConsumerTest() { - // TODO: make tests here + public void kafkaConsumerTest() throws InterruptedException { + // The mock kafka consumer is enabled in the config. Now it should be possible to implement tests using it. https://www.baeldung.com/kafka-mockconsumer + // This code can be implemented as Main.main() function later. + Config config = null; + try { + config = new Config(); + } catch (IOException e){ + System.out.println("Can't load config: " + e); + System.exit(1); + } catch (IllegalArgumentException e) { + System.out.println("Got invalid config: " + e); + System.exit(1); + } + // LOGGER.info("Running main program"); + KafkaController kafkaController = new KafkaController(config); + kafkaController.run(); // FIXME: java.lang.IllegalStateException: Pattern <[testConsumerTopic]> found no topics. at com.teragrep.cfe_39.consumers.kafka.KafkaController.topicScan(KafkaController.java:146) } @Test From 818005e4a262c127f822fe35c790d94d9bec3974 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Fri, 29 Dec 2023 14:08:57 +0200 Subject: [PATCH 012/146] Fixed issue in Kafka mock consumer processing, continued testing Avro-serialization in Kafka consumer processing. --- .../consumers/kafka/DatabaseOutput.java | 10 +++--- .../consumers/kafka/KafkaController.java | 4 +-- .../kafka/MockKafkaConsumerFactoryTemp.java | 10 ++++-- .../teragrep/cfe_39/KafkaConsumerTest.java | 32 ++++++++++++++++++- 4 files changed, 47 insertions(+), 9 deletions(-) diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java index bf77bbb2..de8df87a 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java @@ -100,9 +100,10 @@ boolean checkSizeTooLarge(long fileSize, RecordOffsetObject lastObject) { // This part closes the writing of now "complete" AVRO-file and stores the file to HDFS. syslogAvroWriter.close(); - try (HDFSWriter writer = new HDFSWriter(config, lastObject)) { + // TODO: RESTORE COMMENT BLOCK AFTER TESTING AVRO. + /*try (HDFSWriter writer = new HDFSWriter(config, lastObject)) { writer.commit(syslogFile); // commits the final AVRO-file to HDFS. - } + }*/ // TODO: Delete AVRO-files that have been committed to HDFS? // This part defines a new empty file to which the new AVRO-serialized records are stored until it again hits the 64M size limit. @@ -235,9 +236,10 @@ public void accept(List recordOffsetObjectList) { try { if (syslogAvroWriter != null) { syslogAvroWriter.close(); - try (HDFSWriter writer = new HDFSWriter(config, lastObject)) { + // TODO: RESTORE COMMENT BLOCK AFTER TESTING AVRO. + /*try (HDFSWriter writer = new HDFSWriter(config, lastObject)) { writer.commit(syslogFile); // commits the final AVRO-file to HDFS. - } + }*/ // TODO: Delete AVRO-files that have been committed to HDFS? } } catch (IOException e) { diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java index 58f5c60c..cd7e29b5 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java @@ -129,8 +129,8 @@ private void createReader(String topic, List topicCounters) throws } private void topicScan(DurationStatistics durationStatistics, List topicCounters) { - Map> listTopics = kafkaConsumer.listTopics(Duration.ofSeconds(60)); // TODO: The listTopics is empty, this means problems in mock kafka. - Pattern topicsRegex = Pattern.compile(config.getQueueTopicPattern()); + Map> listTopics = kafkaConsumer.listTopics(Duration.ofSeconds(60)); // Topics can be fetched from mock consumer if the consumer has been updated separately with the partition info. + Pattern topicsRegex = Pattern.compile(config.getQueueTopicPattern()); // Mock consumer has the partitions in this format: queueTopicPattern=^testConsumerTopic-*$ // Find the topics available in Kafka based on given QueueTopicPattern, both active and in-active. // Check how partitions are handled, need to allow using consumer groups for partition read assignments. aka. load balancing diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java index fb6df204..4b0f58d8 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java @@ -4,13 +4,13 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.MockConsumer; import org.apache.kafka.clients.consumer.OffsetResetStrategy; +import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.nio.charset.StandardCharsets; -import java.util.Collections; -import java.util.HashMap; +import java.util.*; /** *

Mock Kafka Consumer Factory

@@ -157,6 +157,12 @@ public static Consumer getConsumer() { endOffsets.put(topicPartition, 14L); // end is exclusive consumer.updateEndOffsets(endOffsets); LOGGER.debug(endOffsets.toString()); + + // consumer.updatePartitions() function must be used so the consumer.listTopics() function can work and return the list of topics properly. + List mockPartitionInfo = new ArrayList<>(); + mockPartitionInfo.add(new PartitionInfo("testConsumerTopic", 0, null, null, null)); + consumer.updatePartitions("testConsumerTopic", mockPartitionInfo); + return consumer; } } \ No newline at end of file diff --git a/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java b/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java index 496e2f75..76ae5151 100644 --- a/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java +++ b/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java @@ -1,8 +1,15 @@ package com.teragrep.cfe_39; import com.teragrep.cfe_39.consumers.kafka.KafkaController; +import org.apache.avro.file.DataFileReader; +import org.apache.avro.io.DatumReader; +import org.apache.avro.specific.SpecificDatumReader; import org.junit.jupiter.api.Test; +import com.teragrep.cfe_39.avro.SyslogRecord; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.io.File; import java.io.IOException; import java.util.Properties; @@ -25,7 +32,30 @@ public void kafkaConsumerTest() throws InterruptedException { } // LOGGER.info("Running main program"); KafkaController kafkaController = new KafkaController(config); - kafkaController.run(); // FIXME: java.lang.IllegalStateException: Pattern <[testConsumerTopic]> found no topics. at com.teragrep.cfe_39.consumers.kafka.KafkaController.topicScan(KafkaController.java:146) + kafkaController.run(); // TODO: Everything is working until kafkaController.topicScan(). AVRO serialization also seems to work well. Now moving to implementing HDFS database and testing it. + } + + // Tests the serialization of the AVRO-file generated in kafkaConsumerTest(). Pathname depends on the configurations set in application.properties file. + @Test + public void AVROReaderTest() throws IOException { + // Deserialize Users from disk + Config config = new Config(); + Path queueDirectory = Paths.get(config.getQueueDirectory()); + File syslogFile = new File( + queueDirectory.toAbsolutePath() + + File.separator + + config.getQueueNamePrefix() + + "." + + 1 + );; + DatumReader userDatumReader = new SpecificDatumReader<>(SyslogRecord.class); + try (DataFileReader dataFileReader = new DataFileReader<>(syslogFile, userDatumReader)) { + SyslogRecord user = null; + while (dataFileReader.hasNext()) { + user = dataFileReader.next(user); + System.out.println(user); + } + } } @Test From 0bfe935ee5e0ceda1cd0e1e0a3d4d050d2dd7d2b Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Wed, 3 Jan 2024 15:41:15 +0200 Subject: [PATCH 013/146] Added Kerberos authentication to HDFS access. --- .../cfe_39/consumers/kafka/HDFSWriter.java | 51 +++++++++++++------ 1 file changed, 35 insertions(+), 16 deletions(-) diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java index dd71c7db..105774c3 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java @@ -7,6 +7,7 @@ import org.apache.hadoop.fs.LocalFileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.apache.hadoop.security.UserGroupInformation; import java.io.File; import java.io.IOException; @@ -20,32 +21,49 @@ public class HDFSWriter implements AutoCloseable{ private final String path; private final FileSystem fs; - public HDFSWriter(Config config, RecordOffsetObject lastObject) { + public HDFSWriter(Config config, RecordOffsetObject lastObject) throws IOException { // Code for initializing the class // Also remember to implement Kerberized access to HDFS. String hdfsuri = config.getHdfsuri(); // Get from config. // The filepath should be something like hdfs:///opt/teragrep/cfe_39/srv/topic_name/0.12345 where 12345 is offset and 0 the partition. - // These values should be fetched from config and other input parameters (topic+partition+offset). + // The values are fetched from config and input parameters (topic+partition+offset). path = config.getHdfsPath()+"/"+lastObject.topic; fileName = lastObject.partition+"."+lastObject.offset; // filename should be constructed from partition and offset. - // ====== Init HDFS File System Object - Configuration conf = new Configuration(); - // Set FileSystem URI - conf.set("fs.defaultFS", hdfsuri); - // Because of Maven - conf.set("fs.hdfs.impl", DistributedFileSystem.class.getName()); - conf.set("fs.file.impl", LocalFileSystem.class.getName()); + // Set HADOOP user here, Kerberus parameters most likely needs to be added here too. - System.setProperty("HADOOP_USER_NAME", "hdfs"); - System.setProperty("hadoop.home.dir", "/"); + System.setProperty("HADOOP_USER_NAME", "hdfs"); // TODO: Add to Config.java + System.setProperty("hadoop.home.dir", "/"); // TODO: Add to Config.java + + // set kerberos host and realm + System.setProperty("java.security.krb5.realm", "DRB.COM"); // TODO: Add to Config.java + System.setProperty("java.security.krb5.kdc", "192.168.33.10"); // TODO: Add to Config.java + + Configuration conf = new Configuration(); + + // enable kerberus + conf.set("hadoop.security.authentication", "kerberos"); // TODO: Add to Config.java + conf.set("hadoop.security.authorization", "true"); // TODO: Add to Config.java + + conf.set("fs.defaultFS", "hdfs://192.168.33.10"); // Set FileSystem URI // TODO: Add to Config.java + conf.set("fs.hdfs.impl", DistributedFileSystem.class.getName()); // Maven stuff? + conf.set("fs.file.impl", LocalFileSystem.class.getName()); // Maven stuff? + + // hack for running locally with fake DNS records + // set this to true if overriding the host name in /etc/hosts + conf.set("dfs.client.use.datanode.hostname", "true"); // TODO: Add to Config.java + + // server principal + // the kerberos principle that the namenode is using + conf.set("dfs.namenode.kerberos.principal.pattern", "hduser/*@DRB.COM"); // TODO: Add to Config.java + + // set usergroup stuff + UserGroupInformation.setConfiguration(conf); + UserGroupInformation.loginUserFromKeytab("dbathgate@DRB.COM", "src/main/resources/dbathgate.keytab"); // TODO: Add to Config.java + // filesystem for HDFS access is set here - try { - fs = FileSystem.get(URI.create(hdfsuri), conf); - } catch (IOException e) { - throw new RuntimeException(e); - } + fs = FileSystem.get(conf); } // Method for committing the AVRO-file to HDFS @@ -80,6 +98,7 @@ public void commit(File syslogFile) { } } + // try-with-resources handles closing the filesystem automatically. public void close() { try { fs.close(); From de78d6391e80f3c45ec373b7f0b13bee84b799b0 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Thu, 4 Jan 2024 15:59:01 +0200 Subject: [PATCH 014/146] Added configurations for Kerberos authentication to HDFS access. --- src/main/java/com/teragrep/cfe_39/Config.java | 43 +++++++++++++++++++ .../cfe_39/consumers/kafka/HDFSWriter.java | 20 ++++----- 2 files changed, 53 insertions(+), 10 deletions(-) diff --git a/src/main/java/com/teragrep/cfe_39/Config.java b/src/main/java/com/teragrep/cfe_39/Config.java index 4f81a496..cfbf38d2 100644 --- a/src/main/java/com/teragrep/cfe_39/Config.java +++ b/src/main/java/com/teragrep/cfe_39/Config.java @@ -37,6 +37,15 @@ public class Config { private final String hdfsuri; private final String queueDirectory; private final String queueNamePrefix; + private final String kerberosHost; + private final String kerberosRealm; + private final String kerberosPrincipal; + private final String hadoopAuthentication; + private final String hadoopAuthorization; + private final String kerberosKeytabUser; + private final String kerberosKeytabPath; + private final String kerberosTestMode; + Config() throws IOException { Properties properties = new Properties(); @@ -89,6 +98,16 @@ public class Config { this.queueDirectory = properties.getProperty("queueDirectory", ""); this.queueNamePrefix = properties.getProperty("queueNamePrefix", ""); + // kerberos + this.kerberosHost = properties.getProperty("java.security.krb5.kdc", ""); + this.kerberosRealm = properties.getProperty("java.security.krb5.realm", ""); + this.hadoopAuthentication = properties.getProperty("hadoop.security.authentication", ""); + this.hadoopAuthorization = properties.getProperty("hadoop.security.authorization", ""); + this.kerberosPrincipal = properties.getProperty("dfs.namenode.kerberos.principal.pattern", ""); + this.kerberosKeytabUser = properties.getProperty("KerberosKeytabUser", ""); + this.kerberosKeytabPath = properties.getProperty("KerberosKeytabPath", ""); + this.kerberosTestMode = properties.getProperty("dfs.client.use.datanode.hostname", "false"); + // kafka this.queueTopicPattern = properties.getProperty("queueTopicPattern", "^.*$"); @@ -185,4 +204,28 @@ public int getDropPartitionsOlderThanHours() { public int getCreatePartitionsInAdvanceHours() { return createPartitionsInAdvanceHours; } + public String getKerberosHost() { + return kerberosHost; + } + public String getKerberosRealm() { + return kerberosRealm; + } + public String getKerberosPrincipal() { + return kerberosPrincipal; + } + public String getHadoopAuthentication() { + return hadoopAuthentication; + } + public String getHadoopAuthorization() { + return hadoopAuthorization; + } + public String getKerberosKeytabUser() { + return kerberosKeytabUser; + } + public String getKerberosKeytabPath() { + return kerberosKeytabPath; + } + public String getKerberosTestMode() { + return kerberosTestMode; + } } \ No newline at end of file diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java index 105774c3..eeb5442a 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java @@ -33,34 +33,34 @@ public HDFSWriter(Config config, RecordOffsetObject lastObject) throws IOExcepti // Set HADOOP user here, Kerberus parameters most likely needs to be added here too. - System.setProperty("HADOOP_USER_NAME", "hdfs"); // TODO: Add to Config.java - System.setProperty("hadoop.home.dir", "/"); // TODO: Add to Config.java + System.setProperty("HADOOP_USER_NAME", "hdfs"); // TODO: not needed because user authentication is done by kerberos? + System.setProperty("hadoop.home.dir", "/"); // TODO: not needed because user authentication is done by kerberos? // set kerberos host and realm - System.setProperty("java.security.krb5.realm", "DRB.COM"); // TODO: Add to Config.java - System.setProperty("java.security.krb5.kdc", "192.168.33.10"); // TODO: Add to Config.java + System.setProperty("java.security.krb5.realm", config.getKerberosRealm()); // DONE: Add to Config.java + System.setProperty("java.security.krb5.kdc", config.getKerberosHost()); // DONE: Add to Config.java Configuration conf = new Configuration(); // enable kerberus - conf.set("hadoop.security.authentication", "kerberos"); // TODO: Add to Config.java - conf.set("hadoop.security.authorization", "true"); // TODO: Add to Config.java + conf.set("hadoop.security.authentication", config.getHadoopAuthentication()); // DONE: Add to Config.java + conf.set("hadoop.security.authorization", config.getHadoopAuthorization()); // DONE: Add to Config.java - conf.set("fs.defaultFS", "hdfs://192.168.33.10"); // Set FileSystem URI // TODO: Add to Config.java + conf.set("fs.defaultFS", hdfsuri); // Set FileSystem URI conf.set("fs.hdfs.impl", DistributedFileSystem.class.getName()); // Maven stuff? conf.set("fs.file.impl", LocalFileSystem.class.getName()); // Maven stuff? // hack for running locally with fake DNS records // set this to true if overriding the host name in /etc/hosts - conf.set("dfs.client.use.datanode.hostname", "true"); // TODO: Add to Config.java + conf.set("dfs.client.use.datanode.hostname", config.getKerberosTestMode()); // DONE: Add to Config.java // server principal // the kerberos principle that the namenode is using - conf.set("dfs.namenode.kerberos.principal.pattern", "hduser/*@DRB.COM"); // TODO: Add to Config.java + conf.set("dfs.namenode.kerberos.principal.pattern", config.getKerberosPrincipal()); // DONE: Add to Config.java // set usergroup stuff UserGroupInformation.setConfiguration(conf); - UserGroupInformation.loginUserFromKeytab("dbathgate@DRB.COM", "src/main/resources/dbathgate.keytab"); // TODO: Add to Config.java + UserGroupInformation.loginUserFromKeytab(config.getKerberosKeytabUser(), config.getKerberosKeytabPath()); // DONE: Add to Config.java // filesystem for HDFS access is set here fs = FileSystem.get(conf); From 0b156004e3fbc0c3da924ffdd6f19a9ce92f3711 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Fri, 5 Jan 2024 15:57:02 +0200 Subject: [PATCH 015/146] Added support to define mock data amount. Debugging AVRO-serialization of records to specific sized files. --- .../consumers/kafka/DatabaseOutput.java | 21 +++-- .../cfe_39/consumers/kafka/HDFSWriter.java | 4 +- .../kafka/MockKafkaConsumerFactoryTemp.java | 91 ++++++++++--------- 3 files changed, 64 insertions(+), 52 deletions(-) diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java index de8df87a..53da2294 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java @@ -71,8 +71,8 @@ public class DatabaseOutput implements Consumer> { this.table = table; this.runtimeStatistics = runtimeStatistics; this.topicCounter = topicCounter; - this.minimumFreeSpace = 32000000; // TODO: CHECK RIGHT VALUE FOR minimumFreeSpace - this.maximumFileSize = 64000000; // TODO: CHECK RIGHT VALUE FOR maximumFileSize. Maximum file size should be 64M. + this.minimumFreeSpace = 32000; // TODO: CHECK RIGHT VALUE FOR minimumFreeSpace + this.maximumFileSize = 64000; // TODO: CHECK RIGHT VALUE FOR maximumFileSize. Maximum file size should be 64M. // queueDirectory and queueNamePrefix shouldn't be critical to name according to the HDFS requirements (topic+partition+offset for filename) as it's just used for storing the AVRO-serialized files. this.writableQueue = new WritableQueue( @@ -103,8 +103,9 @@ boolean checkSizeTooLarge(long fileSize, RecordOffsetObject lastObject) { // TODO: RESTORE COMMENT BLOCK AFTER TESTING AVRO. /*try (HDFSWriter writer = new HDFSWriter(config, lastObject)) { writer.commit(syslogFile); // commits the final AVRO-file to HDFS. - }*/ - // TODO: Delete AVRO-files that have been committed to HDFS? + } + syslogFile.delete(); // Delete AVRO-files that have been committed to HDFS + */ // This part defines a new empty file to which the new AVRO-serialized records are stored until it again hits the 64M size limit. File syslogFile = @@ -217,11 +218,18 @@ public void accept(List recordOffsetObjectList) { // Calculate the size of syslogRecord that is going to be written to syslogAvroWriter-file. long capacity = syslogRecord.toByteBuffer().capacity(); + long capacitybefore = syslogAvroWriter.getFileSize(); // TODO: NOT WORKING PROPERLY! DOESN'T GIVE THE RIGHT FILE SIZE // Check if there is still room in syslogAvroWriter for another syslogRecord. Commit syslogAvroWriter to HDFS if no room left, emptying it out in the process. checkSizeTooLarge(syslogAvroWriter.getFileSize() + capacity, lastObject); // Add syslogRecord to syslogAvroWriter which has rooom for new syslogRecord. syslogAvroWriter.write(syslogRecord); + long capacityafter = syslogAvroWriter.getFileSize(); // TODO: NOT WORKING PROPERLY! DOESN'T GIVE THE RIGHT FILE SIZE + System.out.println("record capacity: " + capacity); + System.out.println("file capacity before adding record: " + capacitybefore); + System.out.println("file capacity after adding record: " + capacityafter); + + /*new RFC5424Timestamp(rfc5424Frame.timestamp).toZonedDateTime().toInstant().getEpochSecond(); rfc5424Frame.appName.toString(); rfc5424Frame.hostname.toString(); @@ -239,8 +247,9 @@ public void accept(List recordOffsetObjectList) { // TODO: RESTORE COMMENT BLOCK AFTER TESTING AVRO. /*try (HDFSWriter writer = new HDFSWriter(config, lastObject)) { writer.commit(syslogFile); // commits the final AVRO-file to HDFS. - }*/ - // TODO: Delete AVRO-files that have been committed to HDFS? + } + syslogFile.delete(); // Delete AVRO-files that have been committed to HDFS + */ } } catch (IOException e) { throw new RuntimeException(e); diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java index eeb5442a..f19d4e13 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java @@ -33,8 +33,8 @@ public HDFSWriter(Config config, RecordOffsetObject lastObject) throws IOExcepti // Set HADOOP user here, Kerberus parameters most likely needs to be added here too. - System.setProperty("HADOOP_USER_NAME", "hdfs"); // TODO: not needed because user authentication is done by kerberos? - System.setProperty("hadoop.home.dir", "/"); // TODO: not needed because user authentication is done by kerberos? + // System.setProperty("HADOOP_USER_NAME", "hdfs"); // TODO: not needed because user authentication is done by kerberos? + // System.setProperty("hadoop.home.dir", "/"); // TODO: not needed because user authentication is done by kerberos? // set kerberos host and realm System.setProperty("java.security.krb5.realm", config.getKerberosRealm()); // DONE: Add to Config.java diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java index 4b0f58d8..05564dec 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java @@ -26,111 +26,111 @@ public class MockKafkaConsumerFactoryTemp { private MockKafkaConsumerFactoryTemp() { } - private static void generateEvents(MockConsumer consumer) { - consumer.addRecord(new ConsumerRecord<>("testConsumerTopic", - 0, + private static void generateEvents(MockConsumer consumer, String topicName, int partition) { + consumer.addRecord(new ConsumerRecord<>(topicName, + partition, 0L, "2022-04-25T07:34:50.804Z".getBytes(StandardCharsets.UTF_8), "<12>1 2022-04-25T07:34:50.804Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"835bf792-91cf-44e3-976b-518330bb8fd3\" source=\"source\" unixtime=\"1650872090805\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872090\"] [WARN] 2022-04-25 07:34:50,804 com.teragrep.jla_02.Log4j Log - Log4j warn says hi!".getBytes(StandardCharsets.UTF_8) ) ); - consumer.addRecord(new ConsumerRecord<>("testConsumerTopic", - 0, + consumer.addRecord(new ConsumerRecord<>(topicName, + partition, 1L, "2022-04-25T07:34:50.806Z".getBytes(StandardCharsets.UTF_8), "<12>1 2022-04-25T07:34:50.806Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"c3f13f9a-05e2-41bd-b0ad-1eca6fd6fd9a\" source=\"source\" unixtime=\"1650872090806\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872090\"] [ERROR] 2022-04-25 07:34:50,806 com.teragrep.jla_02.Log4j Log - Log4j error says hi!".getBytes(StandardCharsets.UTF_8) ) ); - consumer.addRecord(new ConsumerRecord<>("testConsumerTopic", - 0, + consumer.addRecord(new ConsumerRecord<>(topicName, + partition, 2L, "2022-04-25T07:34:50.822Z".getBytes(StandardCharsets.UTF_8), "<12>1 2022-04-25T07:34:50.822Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02\"][event_id@48577 hostname=\"jla-02\" uuid=\"1848d8a1-2f08-4a1e-bec4-ff9e6dd92553\" source=\"source\" unixtime=\"1650872090822\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872090\"] 470647 [Thread-3] INFO com.teragrep.jla_02.Logback Daily - Logback-daily says hi.".getBytes(StandardCharsets.UTF_8) ) ); - consumer.addRecord(new ConsumerRecord<>("testConsumerTopic", - 0, + consumer.addRecord(new ConsumerRecord<>(topicName, + partition, 3L, "2022-04-25T07:34:50.822Z".getBytes(StandardCharsets.UTF_8), "<12>1 2022-04-25T07:34:50.822Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02\"][event_id@48577 hostname=\"jla-02\" uuid=\"5e1a0398-c2a0-468d-a562-c3bb31f0f853\" source=\"source\" unixtime=\"1650872090822\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872090\"] 470646 [Thread-3] INFO com.teragrep.jla_02.Logback Audit - Logback-audit says hi.".getBytes(StandardCharsets.UTF_8) ) ); - consumer.addRecord(new ConsumerRecord<>("testConsumerTopic", - 0, + consumer.addRecord(new ConsumerRecord<>(topicName, + partition, 4L, "2022-04-25T07:34:50.822Z".getBytes(StandardCharsets.UTF_8), "<12>1 2022-04-25T07:34:50.822Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02\"][event_id@48577 hostname=\"jla-02\" uuid=\"6268c3a2-5bda-427f-acce-29416eb445f4\" source=\"source\" unixtime=\"1650872090822\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872090\"] 470647 [Thread-3] INFO com.teragrep.jla_02.Logback Metric - Logback-metric says hi.".getBytes(StandardCharsets.UTF_8) ) ); - consumer.addRecord(new ConsumerRecord<>("testConsumerTopic", - 0, + consumer.addRecord(new ConsumerRecord<>(topicName, + partition, 5L, "2022-04-25T07:34:52.238Z".getBytes(StandardCharsets.UTF_8), "<12>1 2022-04-25T07:34:52.238Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"b500dcaf-1101-4000-b6b9-bfb052ddbf86\" source=\"source\" unixtime=\"1650872092238\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.238 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info audit says hi!]".getBytes(StandardCharsets.UTF_8) ) ); - consumer.addRecord(new ConsumerRecord<>("testConsumerTopic", - 0, + consumer.addRecord(new ConsumerRecord<>(topicName, + partition, 6L, "2022-04-25T07:34:52.239Z".getBytes(StandardCharsets.UTF_8), "<12>1 2022-04-25T07:34:52.239Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"05363122-51ac-4c0b-a681-f5868081f56d\" source=\"source\" unixtime=\"1650872092239\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info daily says hi!]".getBytes(StandardCharsets.UTF_8) ) ); - consumer.addRecord(new ConsumerRecord<>("testConsumerTopic", - 0, + consumer.addRecord(new ConsumerRecord<>(topicName, + partition, 7L, "2022-04-25T07:34:52.239Z".getBytes(StandardCharsets.UTF_8), "<12>1 2022-04-25T07:34:52.239Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"7bbcd843-b795-4c14-b4a1-95f5d445cbcd\" source=\"source\" unixtime=\"1650872092239\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info metric says hi!]".getBytes(StandardCharsets.UTF_8) ) ); - consumer.addRecord(new ConsumerRecord<>("testConsumerTopic", - 0, + consumer.addRecord(new ConsumerRecord<>(topicName, + partition, 8L, "2022-04-25T07:34:52.240Z".getBytes(StandardCharsets.UTF_8), "<12>1 2022-04-25T07:34:52.240Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"2bc0a9f9-237d-4656-b40a-3038aace37f0\" source=\"source\" unixtime=\"1650872092240\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn audit says hi!]".getBytes(StandardCharsets.UTF_8) ) ); - consumer.addRecord(new ConsumerRecord<>("testConsumerTopic", - 0, + consumer.addRecord(new ConsumerRecord<>(topicName, + partition, 9L, "2022-04-25T07:34:52.240Z".getBytes(StandardCharsets.UTF_8), "<12>1 2022-04-25T07:34:52.240Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"ecf61e8d-e3a7-48ef-9b73-3c5a5243d2e6\" source=\"source\" unixtime=\"1650872092240\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn daily says hi!]".getBytes(StandardCharsets.UTF_8) ) ); - consumer.addRecord(new ConsumerRecord<>("testConsumerTopic", - 0, + consumer.addRecord(new ConsumerRecord<>(topicName, + partition, 10L, "2022-04-25T07:34:52.241Z".getBytes(StandardCharsets.UTF_8), "<12>1 2022-04-25T07:34:52.241Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"bf101d5a-e816-4f51-b132-97f8e3431f8e\" source=\"source\" unixtime=\"1650872092241\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.241 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn metric says hi!]".getBytes(StandardCharsets.UTF_8) ) ); - consumer.addRecord(new ConsumerRecord<>("testConsumerTopic", - 0, + consumer.addRecord(new ConsumerRecord<>(topicName, + partition, 11L, "2022-04-25T07:34:52.241Z".getBytes(StandardCharsets.UTF_8), "<12>1 2022-04-25T07:34:52.241Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"ef94d9e9-3c44-4892-b5a6-bf361d13ff97\" source=\"source\" unixtime=\"1650872092241\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.241 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error audit says hi!]".getBytes(StandardCharsets.UTF_8) ) ); - consumer.addRecord(new ConsumerRecord<>("testConsumerTopic", - 0, + consumer.addRecord(new ConsumerRecord<>(topicName, + partition, 12L, "2022-04-25T07:34:52.242Z".getBytes(StandardCharsets.UTF_8), "<12>1 2022-04-25T07:34:52.242Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"5bce6e3d-767d-44b4-a044-6c4872f8f2b5\" source=\"source\" unixtime=\"1650872092242\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.242 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error daily says hi!]".getBytes(StandardCharsets.UTF_8) ) ); - consumer.addRecord(new ConsumerRecord<>("testConsumerTopic", - 0, + consumer.addRecord(new ConsumerRecord<>(topicName, + partition, 13L, "2022-04-25T07:34:52.243Z".getBytes(StandardCharsets.UTF_8), "<12>1 2022-04-25T07:34:52.243Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"3bb55ce4-0ea7-413a-b403-28b174d7ac99\" source=\"source\" unixtime=\"1650872092243\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.243 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error metric says hi!]".getBytes(StandardCharsets.UTF_8) @@ -139,30 +139,33 @@ private static void generateEvents(MockConsumer consumer) { } public static Consumer getConsumer() { + LOGGER.warn("useMockKafkaConsumer is set, using MockKafkaConsumer"); + int amountofloops = 1; // number of loops for adding partitions/records to the mock consumer topic. Each loop adds a new partition of 14 records. final MockConsumer consumer; - - final TopicPartition topicPartition = new TopicPartition("testConsumerTopic", 0); - consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); - consumer.assign(Collections.singletonList(topicPartition)); - + List topicPartitions = new ArrayList<>(); HashMap beginningOffsets = new HashMap<>(); - beginningOffsets.put(topicPartition, 0L); // start is inclusive + HashMap endOffsets = new HashMap<>(); + List mockPartitionInfo = new ArrayList<>(); + // generate the topic partitions and metadata first + for (int i = 0; i < amountofloops; i++) { + TopicPartition topicPartition = new TopicPartition("testConsumerTopic", i); + topicPartitions.add(topicPartition); + beginningOffsets.put(topicPartition, 0L); + endOffsets.put(topicPartition, 14L); + mockPartitionInfo.add(new PartitionInfo("testConsumerTopic", i, null, null, null)); + } + consumer.assign(topicPartitions); consumer.updateBeginningOffsets(beginningOffsets); - generateEvents(consumer); + //insert stuff + for (TopicPartition a : topicPartitions) { + generateEvents(consumer, a.topic(), a.partition()); + } - HashMap endOffsets = new HashMap<>(); - endOffsets.put(topicPartition, 14L); // end is exclusive consumer.updateEndOffsets(endOffsets); - LOGGER.debug(endOffsets.toString()); - - // consumer.updatePartitions() function must be used so the consumer.listTopics() function can work and return the list of topics properly. - List mockPartitionInfo = new ArrayList<>(); - mockPartitionInfo.add(new PartitionInfo("testConsumerTopic", 0, null, null, null)); consumer.updatePartitions("testConsumerTopic", mockPartitionInfo); - return consumer; } } \ No newline at end of file From df067441deab5ea35fbcd3733be359dacca5e615 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Mon, 8 Jan 2024 13:17:52 +0200 Subject: [PATCH 016/146] Debugging issue on how AVRO-file size is tracked. --- .../com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java | 4 ++-- .../cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java | 2 +- .../teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java | 5 ++++- 3 files changed, 7 insertions(+), 4 deletions(-) diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java index 53da2294..5b75168e 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java @@ -71,8 +71,8 @@ public class DatabaseOutput implements Consumer> { this.table = table; this.runtimeStatistics = runtimeStatistics; this.topicCounter = topicCounter; - this.minimumFreeSpace = 32000; // TODO: CHECK RIGHT VALUE FOR minimumFreeSpace - this.maximumFileSize = 64000; // TODO: CHECK RIGHT VALUE FOR maximumFileSize. Maximum file size should be 64M. + this.minimumFreeSpace = 32000000; // TODO: CHECK RIGHT VALUE FOR minimumFreeSpace + this.maximumFileSize = 64000000; // Maximum file size should be 64M (64000000). // queueDirectory and queueNamePrefix shouldn't be critical to name according to the HDFS requirements (topic+partition+offset for filename) as it's just used for storing the AVRO-serialized files. this.writableQueue = new WritableQueue( diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java index 05564dec..2118af81 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java @@ -141,7 +141,7 @@ private static void generateEvents(MockConsumer consumer, String public static Consumer getConsumer() { LOGGER.warn("useMockKafkaConsumer is set, using MockKafkaConsumer"); - int amountofloops = 1; // number of loops for adding partitions/records to the mock consumer topic. Each loop adds a new partition of 14 records. + int amountofloops = 1; // number of loops for adding partitions/records to the mock consumer topic. Each loop adds a new partition of 14 records. 20000 loops results in file size above 64M final MockConsumer consumer; consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); List topicPartitions = new ArrayList<>(); diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java index 99be99fa..313c905a 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java @@ -46,7 +46,10 @@ class SyslogAvroWriter implements AutoCloseable { void write(SyslogRecord syslogRecord) throws IOException{ dataFileWriter.append(syslogRecord); - // dataFileWriter.flush(); // TODO: Avro files 'flush' must be called as few times as possible. Check memory usage impact. Use only automatic flush which is triggered when .close() is called? + dataFileWriter.flush(); // FIXME: getFileSize() doesn't work properly if dataFileWriter.flush() is not called after appending the new record to the AVRO-file. + + // TODO: Avro files 'flush' must be called as few times as possible. Check memory usage impact. Use only automatic flush which is triggered when .close() is called. + // To use the automatic flush AND have a working getFileSize(), the file size must be tracked separately. Approximate the file size by adding the original file size before any appending to the sum of record sizes. } public void close() throws IOException { From 8e58000c435d6a001ed2b187c36f10d6d2041c73 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Tue, 9 Jan 2024 11:07:21 +0200 Subject: [PATCH 017/146] Changed AVRO-serialization to use approximated file size values instead of definite values as definite requires continuous use of flush() after appending records to AVRO. --- .../consumers/kafka/DatabaseOutput.java | 22 ++++++++----------- .../kafka/MockKafkaConsumerFactoryTemp.java | 2 +- .../consumers/kafka/SyslogAvroWriter.java | 6 ++--- .../teragrep/cfe_39/KafkaConsumerTest.java | 5 ++++- 4 files changed, 17 insertions(+), 18 deletions(-) diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java index 5b75168e..4af0a9c5 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java @@ -60,6 +60,7 @@ public class DatabaseOutput implements Consumer> { private final SDVector originHostname; private File syslogFile; private final Config config; + private long approximatedSize; DatabaseOutput( Config config, @@ -72,7 +73,7 @@ public class DatabaseOutput implements Consumer> { this.runtimeStatistics = runtimeStatistics; this.topicCounter = topicCounter; this.minimumFreeSpace = 32000000; // TODO: CHECK RIGHT VALUE FOR minimumFreeSpace - this.maximumFileSize = 64000000; // Maximum file size should be 64M (64000000). + this.maximumFileSize = 60800000; // Maximum file size should be 64M (64000000). 60800000 is 95% of 64M which is a good approximation point. // queueDirectory and queueNamePrefix shouldn't be critical to name according to the HDFS requirements (topic+partition+offset for filename) as it's just used for storing the AVRO-serialized files. this.writableQueue = new WritableQueue( @@ -90,6 +91,7 @@ public class DatabaseOutput implements Consumer> { this.eventNodeSourceHostname = new SDVector("event_node_source@48577","hostname"); this.eventNodeRelayHostname = new SDVector("event_node_relay@48577","hostname"); this.originHostname = new SDVector("origin@48577","hostname"); + this.approximatedSize = 0; } boolean checkSizeTooLarge(long fileSize, RecordOffsetObject lastObject) { @@ -111,6 +113,7 @@ boolean checkSizeTooLarge(long fileSize, RecordOffsetObject lastObject) { File syslogFile = writableQueue.getNextWritableFile(); syslogAvroWriter = new SyslogAvroWriter(syslogFile); + approximatedSize = syslogAvroWriter.getFileSize(); // resets the size approximation. return true; } } catch (IOException ioException) { @@ -158,15 +161,12 @@ public void accept(List recordOffsetObjectList) { // And the content of the AVRO-serialized file that is going to be stored in HDFS is finalized only when the maximumFileSize has been reached. // This means the HDFS filename is only finalized when the AVRO-serialized file is finalized, because every Kafka-record added to the file is going to change the offset that is going to be used for the filename. syslogAvroWriter = new SyslogAvroWriter(syslogFile); + approximatedSize = syslogAvroWriter.getFileSize(); // resets the size approximation. } catch (IOException ioException) { throw new IllegalArgumentException(ioException); } } else { - try { - checkSizeTooLarge(syslogAvroWriter.getFileSize(), lastObject); - } catch (IOException ioException) { - throw new UncheckedIOException(ioException); - } + checkSizeTooLarge(approximatedSize, lastObject); } byte[] byteArray = recordOffsetObject.record; // loads the byte[] contained in recordOffsetObject.record to byteArray. @@ -218,16 +218,12 @@ public void accept(List recordOffsetObjectList) { // Calculate the size of syslogRecord that is going to be written to syslogAvroWriter-file. long capacity = syslogRecord.toByteBuffer().capacity(); - long capacitybefore = syslogAvroWriter.getFileSize(); // TODO: NOT WORKING PROPERLY! DOESN'T GIVE THE RIGHT FILE SIZE // Check if there is still room in syslogAvroWriter for another syslogRecord. Commit syslogAvroWriter to HDFS if no room left, emptying it out in the process. - checkSizeTooLarge(syslogAvroWriter.getFileSize() + capacity, lastObject); + checkSizeTooLarge(approximatedSize + capacity, lastObject); // Add syslogRecord to syslogAvroWriter which has rooom for new syslogRecord. syslogAvroWriter.write(syslogRecord); - - long capacityafter = syslogAvroWriter.getFileSize(); // TODO: NOT WORKING PROPERLY! DOESN'T GIVE THE RIGHT FILE SIZE - System.out.println("record capacity: " + capacity); - System.out.println("file capacity before adding record: " + capacitybefore); - System.out.println("file capacity after adding record: " + capacityafter); + approximatedSize += capacity; + // The difference between actual and approximate file size is about 2,4 % with 64M files. So setting the MaximumFileSize to 95 % of the target should make things work. /*new RFC5424Timestamp(rfc5424Frame.timestamp).toZonedDateTime().toInstant().getEpochSecond(); diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java index 2118af81..2f6a0596 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java @@ -141,7 +141,7 @@ private static void generateEvents(MockConsumer consumer, String public static Consumer getConsumer() { LOGGER.warn("useMockKafkaConsumer is set, using MockKafkaConsumer"); - int amountofloops = 1; // number of loops for adding partitions/records to the mock consumer topic. Each loop adds a new partition of 14 records. 20000 loops results in file size above 64M + int amountofloops = 17777; // number of loops for adding partitions/records to the mock consumer topic. Each loop adds a new partition of 14 records. 17777 loops results in file size slightly above 64M. final MockConsumer consumer; consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); List topicPartitions = new ArrayList<>(); diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java index 313c905a..e7950ef9 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java @@ -46,10 +46,10 @@ class SyslogAvroWriter implements AutoCloseable { void write(SyslogRecord syslogRecord) throws IOException{ dataFileWriter.append(syslogRecord); - dataFileWriter.flush(); // FIXME: getFileSize() doesn't work properly if dataFileWriter.flush() is not called after appending the new record to the AVRO-file. + // dataFileWriter.flush(); // getFileSize() doesn't work properly if dataFileWriter.flush() is not called after appending a new record to the AVRO-file. - // TODO: Avro files 'flush' must be called as few times as possible. Check memory usage impact. Use only automatic flush which is triggered when .close() is called. - // To use the automatic flush AND have a working getFileSize(), the file size must be tracked separately. Approximate the file size by adding the original file size before any appending to the sum of record sizes. + // Avro files 'flush' must be called as few times as possible. Check memory usage impact. Use only automatic flush which is triggered when .close() is called. + // To use the automatic flush AND have a working getFileSize(), the file size must be tracked separately. Approximate the file size by adding the original file size before any appending to the sum of record sizes. } public void close() throws IOException { diff --git a/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java b/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java index 76ae5151..06c31c3d 100644 --- a/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java +++ b/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java @@ -46,16 +46,19 @@ public void AVROReaderTest() throws IOException { + File.separator + config.getQueueNamePrefix() + "." - + 1 + + 1 // change value if there are more than one avro-file generated etc. );; + int counter = 0; DatumReader userDatumReader = new SpecificDatumReader<>(SyslogRecord.class); try (DataFileReader dataFileReader = new DataFileReader<>(syslogFile, userDatumReader)) { SyslogRecord user = null; while (dataFileReader.hasNext()) { user = dataFileReader.next(user); System.out.println(user); + counter++; } } + System.out.println("Total number of records: " + counter); } @Test From 00d790ecdb9f9cd06072e140ae3c068a56fff15d Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Tue, 9 Jan 2024 14:08:07 +0200 Subject: [PATCH 018/146] Still having issues in the filesize of AVRO-serialization. --- .../teragrep/cfe_39/KafkaConsumerTest.java | 33 +++++++++++-------- 1 file changed, 19 insertions(+), 14 deletions(-) diff --git a/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java b/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java index 06c31c3d..272bf300 100644 --- a/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java +++ b/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java @@ -41,24 +41,29 @@ public void AVROReaderTest() throws IOException { // Deserialize Users from disk Config config = new Config(); Path queueDirectory = Paths.get(config.getQueueDirectory()); - File syslogFile = new File( - queueDirectory.toAbsolutePath() - + File.separator - + config.getQueueNamePrefix() - + "." - + 1 // change value if there are more than one avro-file generated etc. - );; int counter = 0; - DatumReader userDatumReader = new SpecificDatumReader<>(SyslogRecord.class); - try (DataFileReader dataFileReader = new DataFileReader<>(syslogFile, userDatumReader)) { - SyslogRecord user = null; - while (dataFileReader.hasNext()) { - user = dataFileReader.next(user); - System.out.println(user); - counter++; + for (int i = 5; i<=6; i++) { + File syslogFile = new File( + queueDirectory.toAbsolutePath() + + File.separator + + config.getQueueNamePrefix() + + "." + + i + ); + DatumReader userDatumReader = new SpecificDatumReader<>(SyslogRecord.class); + try (DataFileReader dataFileReader = new DataFileReader<>(syslogFile, userDatumReader)) { + SyslogRecord user = null; + while (dataFileReader.hasNext()) { + user = dataFileReader.next(user); + System.out.println(user); + counter++; + } } } System.out.println("Total number of records: " + counter); + // 1-2 (only close): 248878, filesize too small + // 3-4 (flush after append): 248878, filesize ok + // 5-6 (flush only before close) : 248878, filesize too small } @Test From f7e31699bcbaf405af180a29fe5e225b0d15e348 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Wed, 10 Jan 2024 14:22:31 +0200 Subject: [PATCH 019/146] Rolled back to using definitive file size values on avro-serialization files with flush() after every record append. Improved test cases. --- src/main/java/com/teragrep/cfe_39/Config.java | 8 + .../consumers/kafka/DatabaseOutput.java | 14 +- .../consumers/kafka/KafkaController.java | 15 +- .../kafka/MockKafkaConsumerFactoryTemp.java | 2 +- .../consumers/kafka/SyslogAvroWriter.java | 2 +- .../cfe_39/metrics/DurationStatistics.java | 5 + .../teragrep/cfe_39/KafkaConsumerTest.java | 156 +++++++++++++++--- 7 files changed, 169 insertions(+), 33 deletions(-) diff --git a/src/main/java/com/teragrep/cfe_39/Config.java b/src/main/java/com/teragrep/cfe_39/Config.java index cfbf38d2..3967269f 100644 --- a/src/main/java/com/teragrep/cfe_39/Config.java +++ b/src/main/java/com/teragrep/cfe_39/Config.java @@ -45,6 +45,7 @@ public class Config { private final String kerberosKeytabUser; private final String kerberosKeytabPath; private final String kerberosTestMode; + private long maximumFileSize; Config() throws IOException { @@ -121,6 +122,7 @@ public class Config { throw new IOException("File '" + loginConfig + "' set by java.security.auth.login.config does not exist"); } System.setProperty("java.security.auth.login.config", loginConfig); + this.maximumFileSize = 60800000; // default value // Just for loggers to work Path log4j2Config = Paths.get(properties.getProperty("log4j2.configurationFile", System.getProperty("user.dir") + "/etc/log4j2.properties")); @@ -228,4 +230,10 @@ public String getKerberosKeytabPath() { public String getKerberosTestMode() { return kerberosTestMode; } + public long getMaximumFileSize() { + return maximumFileSize; + } + public void setMaximumFileSize(long maximumFileSize) { + this.maximumFileSize = maximumFileSize; + } } \ No newline at end of file diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java index 4af0a9c5..138fbafc 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java @@ -73,7 +73,7 @@ public class DatabaseOutput implements Consumer> { this.runtimeStatistics = runtimeStatistics; this.topicCounter = topicCounter; this.minimumFreeSpace = 32000000; // TODO: CHECK RIGHT VALUE FOR minimumFreeSpace - this.maximumFileSize = 60800000; // Maximum file size should be 64M (64000000). 60800000 is 95% of 64M which is a good approximation point. + this.maximumFileSize = config.getMaximumFileSize();; // Maximum file size should be 64M (64000000). 60800000 is 95% of 64M which is a good approximation point. // queueDirectory and queueNamePrefix shouldn't be critical to name according to the HDFS requirements (topic+partition+offset for filename) as it's just used for storing the AVRO-serialized files. this.writableQueue = new WritableQueue( @@ -166,7 +166,12 @@ public void accept(List recordOffsetObjectList) { throw new IllegalArgumentException(ioException); } } else { - checkSizeTooLarge(approximatedSize, lastObject); + // checkSizeTooLarge(approximatedSize, lastObject); // FIXME: approximatedSize is not working properly without the use of flush() after append. File sizes are all over the place. + try { + checkSizeTooLarge(syslogAvroWriter.getFileSize(), lastObject); + } catch (IOException ioException) { + throw new UncheckedIOException(ioException); + } } byte[] byteArray = recordOffsetObject.record; // loads the byte[] contained in recordOffsetObject.record to byteArray. @@ -219,8 +224,9 @@ public void accept(List recordOffsetObjectList) { // Calculate the size of syslogRecord that is going to be written to syslogAvroWriter-file. long capacity = syslogRecord.toByteBuffer().capacity(); // Check if there is still room in syslogAvroWriter for another syslogRecord. Commit syslogAvroWriter to HDFS if no room left, emptying it out in the process. - checkSizeTooLarge(approximatedSize + capacity, lastObject); - // Add syslogRecord to syslogAvroWriter which has rooom for new syslogRecord. + // checkSizeTooLarge(approximatedSize + capacity, lastObject); // FIXME: approximatedSize is not working properly without the use of flush() after append. File sizes are all over the place. + checkSizeTooLarge(syslogAvroWriter.getFileSize() + capacity, lastObject); + // Add syslogRecord to syslogAvroWriter which has room for new syslogRecord. syslogAvroWriter.write(syslogRecord); approximatedSize += capacity; // The difference between actual and approximate file size is about 2,4 % with 64M files. So setting the MaximumFileSize to 95 % of the target should make things work. diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java index cd7e29b5..22f41656 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java @@ -56,11 +56,14 @@ public class KafkaController { private final List threads = new ArrayList<>(); private final Set activeTopics = new HashSet<>(); private final RuntimeStatistics runtimeStatistics = new RuntimeStatistics(); + private boolean keepRunning; + private boolean useMockKafkaConsumer; public KafkaController(Config config) { + keepRunning = true; this.config = config; Properties readerKafkaProperties = config.getKafkaConsumerProperties(); - boolean useMockKafkaConsumer = Boolean.parseBoolean( + this.useMockKafkaConsumer = Boolean.parseBoolean( readerKafkaProperties.getProperty("useMockKafkaConsumer", "false") ); if (useMockKafkaConsumer) { @@ -85,14 +88,20 @@ public void run() throws InterruptedException { ); topicMetrics.register(); - while (true) { - LOGGER.debug("Scanning for threads"); + while (keepRunning) { + LOGGER.info("Scanning for threads"); topicScan(durationStatistics, topicCounters); // log stuff durationStatistics.log(); long topicScanDelay = 30000L; Thread.sleep(topicScanDelay); + + if (durationStatistics.getTotalRecords() > 0 & useMockKafkaConsumer) { + LOGGER.info("Processed all the test records. Closing."); + keepRunning = false; + } + } } diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java index 2f6a0596..3ea1cf1b 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java @@ -141,7 +141,7 @@ private static void generateEvents(MockConsumer consumer, String public static Consumer getConsumer() { LOGGER.warn("useMockKafkaConsumer is set, using MockKafkaConsumer"); - int amountofloops = 17777; // number of loops for adding partitions/records to the mock consumer topic. Each loop adds a new partition of 14 records. 17777 loops results in file size slightly above 64M. + int amountofloops = 10; // number of loops for adding partitions/records to the mock consumer topic. Each loop adds a new partition of 14 records. 17777 loops results in file size slightly above 64M. 10 loops is sized at 36,102 bits. final MockConsumer consumer; consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); List topicPartitions = new ArrayList<>(); diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java index e7950ef9..7eee088f 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java @@ -46,7 +46,7 @@ class SyslogAvroWriter implements AutoCloseable { void write(SyslogRecord syslogRecord) throws IOException{ dataFileWriter.append(syslogRecord); - // dataFileWriter.flush(); // getFileSize() doesn't work properly if dataFileWriter.flush() is not called after appending a new record to the AVRO-file. + dataFileWriter.flush(); // getFileSize() doesn't work properly if dataFileWriter.flush() is not called after appending a new record to the AVRO-file. // Avro files 'flush' must be called as few times as possible. Check memory usage impact. Use only automatic flush which is triggered when .close() is called. // To use the automatic flush AND have a working getFileSize(), the file size must be tracked separately. Approximate the file size by adding the original file size before any appending to the sum of record sizes. diff --git a/src/main/java/com/teragrep/cfe_39/metrics/DurationStatistics.java b/src/main/java/com/teragrep/cfe_39/metrics/DurationStatistics.java index 684c65d7..b074767b 100644 --- a/src/main/java/com/teragrep/cfe_39/metrics/DurationStatistics.java +++ b/src/main/java/com/teragrep/cfe_39/metrics/DurationStatistics.java @@ -69,6 +69,11 @@ public void report(RuntimeStatistics runtimeStatistics) { lastBytes = currentBytes; } + public long getTotalRecords() { + long test = records.getReader().get(); + return test; + } + public void log() { LOGGER.info( "## TOTAL records <" + records.getReader().get() + "> " + diff --git a/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java b/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java index 272bf300..edd964ba 100644 --- a/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java +++ b/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java @@ -4,8 +4,11 @@ import org.apache.avro.file.DataFileReader; import org.apache.avro.io.DatumReader; import org.apache.avro.specific.SpecificDatumReader; +import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; import com.teragrep.cfe_39.avro.SyslogRecord; + +import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; @@ -14,9 +17,134 @@ import java.util.Properties; public class KafkaConsumerTest { + // Make sure application.properties has consumer.useMockKafkaConsumer=true enabled for Kafka testing. - // TODO: make tests here. Make sure application.properties has consumer.useMockKafkaConsumer=true enabled for Kafka testing. @Test + public void configTest() { + // Configuration tests done, configurations working correctly with the right .jaas and .properties files. + try { + Config config = new Config(); + + Properties readerKafkaProperties = config.getKafkaConsumerProperties(); + + // Test extracting useMockKafkaConsumer value from config. + boolean useMockKafkaConsumer = Boolean.parseBoolean( + readerKafkaProperties.getProperty("useMockKafkaConsumer", "false") + ); + System.out.println("useMockKafkaConsumer: "+useMockKafkaConsumer); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + + @Test + public void avroFullTest() throws InterruptedException { + Config config = null; + try { + config = new Config(); + } catch (IOException e){ + System.out.println("Can't load config: " + e); + System.exit(1); + } catch (IllegalArgumentException e) { + System.out.println("Got invalid config: " + e); + System.exit(1); + } + config.setMaximumFileSize(8000); // 10 loops (140 records) are in use at the moment, and that is sized at 36,102 bits. + KafkaController kafkaController = new KafkaController(config); + kafkaController.run(); + try { + int counter = avroReader(1, 5); + Assertions.assertEquals(140, counter); + } catch (IOException e) { + throw new RuntimeException(e); + } + cleanup(config, 1, 5); + } + + // Reads the data from a list of avro files + public int avroReader(int start, int end) throws IOException { + // Deserialize Users from disk + Config config = new Config(); + Path queueDirectory = Paths.get(config.getQueueDirectory()); + int counter = 0; + int looper = 0; + for (int i = start; i<=end; i++) { + File syslogFile = new File( + queueDirectory.toAbsolutePath() + + File.separator + + config.getQueueNamePrefix() + + "." + + i + ); + DatumReader userDatumReader = new SpecificDatumReader<>(SyslogRecord.class); + try (DataFileReader dataFileReader = new DataFileReader<>(syslogFile, userDatumReader)) { + SyslogRecord user = null; + while (dataFileReader.hasNext()) { + user = dataFileReader.next(user); + System.out.println(user); + counter++; + looper++; + // All the mock data is generated from a set of 14 records. + if (looper <= 1) { + Assertions.assertEquals("[WARN] 2022-04-25 07:34:50,804 com.teragrep.jla_02.Log4j Log - Log4j warn says hi!", user.getMessage().toString()); + } else if (looper == 2) { + Assertions.assertEquals("[ERROR] 2022-04-25 07:34:50,806 com.teragrep.jla_02.Log4j Log - Log4j error says hi!", user.getMessage().toString()); + } else if (looper == 3) { + Assertions.assertEquals("470647 [Thread-3] INFO com.teragrep.jla_02.Logback Daily - Logback-daily says hi.", user.getMessage().toString()); + } else if (looper == 4) { + Assertions.assertEquals("470646 [Thread-3] INFO com.teragrep.jla_02.Logback Audit - Logback-audit says hi.", user.getMessage().toString()); + } else if (looper == 5) { + Assertions.assertEquals("470647 [Thread-3] INFO com.teragrep.jla_02.Logback Metric - Logback-metric says hi.", user.getMessage().toString()); + } else if (looper == 6) { + Assertions.assertEquals("25.04.2022 07:34:52.238 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info audit says hi!]", user.getMessage().toString()); + } else if (looper == 7) { + Assertions.assertEquals("25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info daily says hi!]", user.getMessage().toString()); + } else if (looper == 8) { + Assertions.assertEquals("25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info metric says hi!]", user.getMessage().toString()); + } else if (looper == 9) { + Assertions.assertEquals("25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn audit says hi!]", user.getMessage().toString()); + } else if (looper == 10) { + Assertions.assertEquals("25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn daily says hi!]", user.getMessage().toString()); + } else if (looper == 11) { + Assertions.assertEquals("25.04.2022 07:34:52.241 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn metric says hi!]", user.getMessage().toString()); + } else if (looper == 12) { + Assertions.assertEquals("25.04.2022 07:34:52.241 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error audit says hi!]", user.getMessage().toString()); + } else if (looper == 13) { + Assertions.assertEquals("25.04.2022 07:34:52.242 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error daily says hi!]", user.getMessage().toString()); + } else { + Assertions.assertEquals("25.04.2022 07:34:52.243 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error metric says hi!]", user.getMessage().toString()); + looper = 0; + } + } + } + } + System.out.println("Total number of records: " + counter); + return counter; + } + + // Deletes the avro-files that were created during testing. + public void cleanup(Config config, int start, int end) { + Path queueDirectory = Paths.get(config.getQueueDirectory()); + String queueNamePrefix = config.getQueueNamePrefix(); + for (int nextSequenceNumber = start; nextSequenceNumber <= end; nextSequenceNumber++) { + File file = new File( + queueDirectory.toAbsolutePath() + + File.separator + + queueNamePrefix + + "." + + nextSequenceNumber + ); + try { + boolean result = Files.deleteIfExists(file.toPath()); //surround it in try catch block + } catch (IOException e) { + throw new RuntimeException(e); + } + } + } + + + // @Test public void kafkaConsumerTest() throws InterruptedException { // The mock kafka consumer is enabled in the config. Now it should be possible to implement tests using it. https://www.baeldung.com/kafka-mockconsumer // This code can be implemented as Main.main() function later. @@ -30,13 +158,14 @@ public void kafkaConsumerTest() throws InterruptedException { System.out.println("Got invalid config: " + e); System.exit(1); } + config.setMaximumFileSize(300000); // 10 loops are in use at the moment, and that is sized at 36,102 bits. // LOGGER.info("Running main program"); KafkaController kafkaController = new KafkaController(config); - kafkaController.run(); // TODO: Everything is working until kafkaController.topicScan(). AVRO serialization also seems to work well. Now moving to implementing HDFS database and testing it. + kafkaController.run(); } // Tests the serialization of the AVRO-file generated in kafkaConsumerTest(). Pathname depends on the configurations set in application.properties file. - @Test + // @Test public void AVROReaderTest() throws IOException { // Deserialize Users from disk Config config = new Config(); @@ -61,26 +190,5 @@ public void AVROReaderTest() throws IOException { } } System.out.println("Total number of records: " + counter); - // 1-2 (only close): 248878, filesize too small - // 3-4 (flush after append): 248878, filesize ok - // 5-6 (flush only before close) : 248878, filesize too small - } - - @Test - public void configTest() { - // Configuration tests done, configurations working correctly with the right .jaas and .properties files. - try { - Config config = new Config(); - - Properties readerKafkaProperties = config.getKafkaConsumerProperties(); - - // Test extracting useMockKafkaConsumer value from config. - boolean useMockKafkaConsumer = Boolean.parseBoolean( - readerKafkaProperties.getProperty("useMockKafkaConsumer", "false") - ); - System.out.println("useMockKafkaConsumer: "+useMockKafkaConsumer); - } catch (IOException e) { - throw new RuntimeException(e); - } } } From 7fdc758d4bcd7dd5d314ead076f62b0b9853286b Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Wed, 10 Jan 2024 16:43:46 +0200 Subject: [PATCH 020/146] Setting up HDFS access tests. --- .../consumers/kafka/DatabaseOutput.java | 14 +- .../cfe_39/consumers/kafka/HDFSWriter.java | 146 ++++++++++-------- .../teragrep/cfe_39/KafkaConsumerTest.java | 2 +- 3 files changed, 89 insertions(+), 73 deletions(-) diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java index 138fbafc..cfea4bde 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java @@ -25,9 +25,7 @@ // The kafka stream should first be deserialized using rlo_06 and then serialized again using avro and stored in HDFS. // The target where the record is stored in HDFS is based on the topic, partition and offset. ie. topic_name/0.123456 where offset is 123456 -// TODO: make testcases using MockKafkaConsumerFactory. The mock consumer is activated for testing using the configuration file: readerKafkaProperties.getProperty("useMockKafkaConsumer", "false") -// DONE: -// Thus the configuration file is the first thing to make for testcases, and the first testcases are for testing the configuration file. +// The mock consumer is activated for testing using the configuration file: readerKafkaProperties.getProperty("useMockKafkaConsumer", "false") public class DatabaseOutput implements Consumer> { private static final Logger LOGGER = LoggerFactory.getLogger(DatabaseOutput.class); @@ -102,12 +100,9 @@ boolean checkSizeTooLarge(long fileSize, RecordOffsetObject lastObject) { // This part closes the writing of now "complete" AVRO-file and stores the file to HDFS. syslogAvroWriter.close(); - // TODO: RESTORE COMMENT BLOCK AFTER TESTING AVRO. - /*try (HDFSWriter writer = new HDFSWriter(config, lastObject)) { + try (HDFSWriter writer = new HDFSWriter(config, lastObject)) { writer.commit(syslogFile); // commits the final AVRO-file to HDFS. } - syslogFile.delete(); // Delete AVRO-files that have been committed to HDFS - */ // This part defines a new empty file to which the new AVRO-serialized records are stored until it again hits the 64M size limit. File syslogFile = @@ -246,12 +241,9 @@ public void accept(List recordOffsetObjectList) { try { if (syslogAvroWriter != null) { syslogAvroWriter.close(); - // TODO: RESTORE COMMENT BLOCK AFTER TESTING AVRO. - /*try (HDFSWriter writer = new HDFSWriter(config, lastObject)) { + try (HDFSWriter writer = new HDFSWriter(config, lastObject)) { writer.commit(syslogFile); // commits the final AVRO-file to HDFS. } - syslogFile.delete(); // Delete AVRO-files that have been committed to HDFS - */ } } catch (IOException e) { throw new RuntimeException(e); diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java index f19d4e13..8fcba917 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java @@ -14,96 +14,120 @@ import java.net.URI; import java.nio.file.Files; import java.nio.file.Paths; +import java.util.Properties; public class HDFSWriter implements AutoCloseable{ private final String fileName; private final String path; private final FileSystem fs; + private final boolean useMockKafkaConsumer; // test-mode switch public HDFSWriter(Config config, RecordOffsetObject lastObject) throws IOException { - // Code for initializing the class - // Also remember to implement Kerberized access to HDFS. - String hdfsuri = config.getHdfsuri(); // Get from config. - // The filepath should be something like hdfs:///opt/teragrep/cfe_39/srv/topic_name/0.12345 where 12345 is offset and 0 the partition. - // The values are fetched from config and input parameters (topic+partition+offset). - path = config.getHdfsPath()+"/"+lastObject.topic; - fileName = lastObject.partition+"."+lastObject.offset; // filename should be constructed from partition and offset. + // Check for testmode from config. + Properties readerKafkaProperties = config.getKafkaConsumerProperties(); + this.useMockKafkaConsumer = Boolean.parseBoolean( + readerKafkaProperties.getProperty("useMockKafkaConsumer", "false") + ); + if (useMockKafkaConsumer) { + // TODO: CODE FOR TEST-MODE GOES HERE! When test-mode is on, use normal HDFS access instead of Kerberized access. The target HDFS server should also be set to be the test-server. + path = config.getHdfsPath() + "/" + lastObject.topic; + fileName = lastObject.partition + "." + lastObject.offset; // filename should be constructed from partition and offset. + fs = null; + }else { + // Code for initializing the class + // Also remember to implement Kerberized access to HDFS. + String hdfsuri = config.getHdfsuri(); // Get from config. - // Set HADOOP user here, Kerberus parameters most likely needs to be added here too. - // System.setProperty("HADOOP_USER_NAME", "hdfs"); // TODO: not needed because user authentication is done by kerberos? - // System.setProperty("hadoop.home.dir", "/"); // TODO: not needed because user authentication is done by kerberos? + // The filepath should be something like hdfs:///opt/teragrep/cfe_39/srv/topic_name/0.12345 where 12345 is offset and 0 the partition. + // The values are fetched from config and input parameters (topic+partition+offset). + path = config.getHdfsPath() + "/" + lastObject.topic; + fileName = lastObject.partition + "." + lastObject.offset; // filename should be constructed from partition and offset. - // set kerberos host and realm - System.setProperty("java.security.krb5.realm", config.getKerberosRealm()); // DONE: Add to Config.java - System.setProperty("java.security.krb5.kdc", config.getKerberosHost()); // DONE: Add to Config.java - Configuration conf = new Configuration(); + // Set HADOOP user here, Kerberus parameters most likely needs to be added here too. + // System.setProperty("HADOOP_USER_NAME", "hdfs"); // Not needed because user authentication is done by kerberos? + // System.setProperty("hadoop.home.dir", "/"); // Not needed because user authentication is done by kerberos? - // enable kerberus - conf.set("hadoop.security.authentication", config.getHadoopAuthentication()); // DONE: Add to Config.java - conf.set("hadoop.security.authorization", config.getHadoopAuthorization()); // DONE: Add to Config.java + // set kerberos host and realm + System.setProperty("java.security.krb5.realm", config.getKerberosRealm()); // DONE: Add to Config.java + System.setProperty("java.security.krb5.kdc", config.getKerberosHost()); // DONE: Add to Config.java - conf.set("fs.defaultFS", hdfsuri); // Set FileSystem URI - conf.set("fs.hdfs.impl", DistributedFileSystem.class.getName()); // Maven stuff? - conf.set("fs.file.impl", LocalFileSystem.class.getName()); // Maven stuff? + Configuration conf = new Configuration(); - // hack for running locally with fake DNS records - // set this to true if overriding the host name in /etc/hosts - conf.set("dfs.client.use.datanode.hostname", config.getKerberosTestMode()); // DONE: Add to Config.java + // enable kerberus + conf.set("hadoop.security.authentication", config.getHadoopAuthentication()); // DONE: Add to Config.java + conf.set("hadoop.security.authorization", config.getHadoopAuthorization()); // DONE: Add to Config.java - // server principal - // the kerberos principle that the namenode is using - conf.set("dfs.namenode.kerberos.principal.pattern", config.getKerberosPrincipal()); // DONE: Add to Config.java + conf.set("fs.defaultFS", hdfsuri); // Set FileSystem URI + conf.set("fs.hdfs.impl", DistributedFileSystem.class.getName()); // Maven stuff? + conf.set("fs.file.impl", LocalFileSystem.class.getName()); // Maven stuff? - // set usergroup stuff - UserGroupInformation.setConfiguration(conf); - UserGroupInformation.loginUserFromKeytab(config.getKerberosKeytabUser(), config.getKerberosKeytabPath()); // DONE: Add to Config.java + // hack for running locally with fake DNS records + // set this to true if overriding the host name in /etc/hosts + conf.set("dfs.client.use.datanode.hostname", config.getKerberosTestMode()); // DONE: Add to Config.java - // filesystem for HDFS access is set here - fs = FileSystem.get(conf); + // server principal + // the kerberos principle that the namenode is using + conf.set("dfs.namenode.kerberos.principal.pattern", config.getKerberosPrincipal()); // DONE: Add to Config.java + + // set usergroup stuff + UserGroupInformation.setConfiguration(conf); + UserGroupInformation.loginUserFromKeytab(config.getKerberosKeytabUser(), config.getKerberosKeytabPath()); // DONE: Add to Config.java + + // filesystem for HDFS access is set here + fs = FileSystem.get(conf); + } } // Method for committing the AVRO-file to HDFS public void commit(File syslogFile) { - //Get the filesystem - HDFS - try { - //==== Create folder if not exists - Path workingDir=fs.getWorkingDirectory(); - // Sets the directory where the data should be stored, if the directory doesn't exist then it's created. - Path newFolderPath= new Path(path); - if(!fs.exists(newFolderPath)) { - // Create new Directory - fs.mkdirs(newFolderPath); - // logger.info("Path "+path+" created."); - } - //==== Write file - // logger.info("Begin Write file into hdfs"); - //Create a path - Path hdfswritepath = new Path(newFolderPath + "/" + fileName); // filename should be set according to the requirements: 0.12345 where 0 is Kafka partition and 12345 is Kafka offset. - //Init output stream - FSDataOutputStream outputStream=fs.create(hdfswritepath); - // Write the file contents of syslogFile to hdfswritepath in HDFS. - // file to bytes[] - byte[] bytes = Files.readAllBytes(Paths.get(syslogFile.getPath())); - outputStream.write(bytes); - outputStream.close(); - // logger.info("End Write file into hdfs"); - - } catch (IOException e) { - throw new RuntimeException(e); + if (useMockKafkaConsumer) { + // TODO: CODE FOR TEST-MODE GOES HERE! + }else { + //Get the filesystem - HDFS + try { + //==== Create folder if not exists + Path workingDir = fs.getWorkingDirectory(); + // Sets the directory where the data should be stored, if the directory doesn't exist then it's created. + Path newFolderPath = new Path(path); + if (!fs.exists(newFolderPath)) { + // Create new Directory + fs.mkdirs(newFolderPath); + // logger.info("Path "+path+" created."); + } + + //==== Write file + // logger.info("Begin Write file into hdfs"); + //Create a path + Path hdfswritepath = new Path(newFolderPath + "/" + fileName); // filename should be set according to the requirements: 0.12345 where 0 is Kafka partition and 12345 is Kafka offset. + //Init output stream + FSDataOutputStream outputStream = fs.create(hdfswritepath); + // Write the file contents of syslogFile to hdfswritepath in HDFS. + // file to bytes[] + byte[] bytes = Files.readAllBytes(Paths.get(syslogFile.getPath())); + outputStream.write(bytes); + outputStream.close(); + // logger.info("End Write file into hdfs"); + boolean delete = syslogFile.delete(); // deletes the avro-file from the local disk now that it has been committed to HDFS. + + } catch (IOException e) { + throw new RuntimeException(e); + } } } // try-with-resources handles closing the filesystem automatically. public void close() { - try { - fs.close(); - } catch (IOException e) { - throw new RuntimeException(e); + if (fs != null) { + try { + fs.close(); + } catch (IOException e) { + throw new RuntimeException(e); + } } } diff --git a/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java b/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java index edd964ba..bfa3fe5d 100644 --- a/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java +++ b/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java @@ -39,7 +39,7 @@ public void configTest() { @Test - public void avroFullTest() throws InterruptedException { + public void kafkaAndAvroFullTest() throws InterruptedException { Config config = null; try { config = new Config(); From cf9ff7321d73e0ff461ae086e0d75a41501b8b5e Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Fri, 12 Jan 2024 15:44:25 +0200 Subject: [PATCH 021/146] Setting up HDFS access tests (2). --- src/main/java/com/teragrep/cfe_39/Config.java | 1 + .../cfe_39/consumers/kafka/HDFSWriter.java | 110 ++++++++++++++---- .../consumers/kafka/KafkaController.java | 2 +- 3 files changed, 92 insertions(+), 21 deletions(-) diff --git a/src/main/java/com/teragrep/cfe_39/Config.java b/src/main/java/com/teragrep/cfe_39/Config.java index 3967269f..f3cd4ba4 100644 --- a/src/main/java/com/teragrep/cfe_39/Config.java +++ b/src/main/java/com/teragrep/cfe_39/Config.java @@ -47,6 +47,7 @@ public class Config { private final String kerberosTestMode; private long maximumFileSize; + // TODO: Set up configuration check for important parameters. Config() throws IOException { Properties properties = new Properties(); diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java index 8fcba917..e0d49585 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java @@ -2,14 +2,12 @@ import com.teragrep.cfe_39.Config; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.LocalFileSystem; -import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.*; import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.security.UserGroupInformation; import java.io.File; +import java.io.FileInputStream; import java.io.IOException; import java.net.URI; import java.nio.file.Files; @@ -23,6 +21,9 @@ public class HDFSWriter implements AutoCloseable{ private final FileSystem fs; private final boolean useMockKafkaConsumer; // test-mode switch + // Create files as whole but stream the contents into them. Avro files 'flush' must be called as few times as possible. Check memory usage impact + // Later make sure to check the avro file flush issue where the file size is all over the place if flush is not used after every append to the file. + public HDFSWriter(Config config, RecordOffsetObject lastObject) throws IOException { // Check for testmode from config. @@ -32,18 +33,41 @@ public HDFSWriter(Config config, RecordOffsetObject lastObject) throws IOExcepti ); if (useMockKafkaConsumer) { - // TODO: CODE FOR TEST-MODE GOES HERE! When test-mode is on, use normal HDFS access instead of Kerberized access. The target HDFS server should also be set to be the test-server. - path = config.getHdfsPath() + "/" + lastObject.topic; - fileName = lastObject.partition + "." + lastObject.offset; // filename should be constructed from partition and offset. - fs = null; + // Code for initializing the class in test mode without kerberos. + String hdfsuri = config.getHdfsuri(); // Get from config. + + // The filepath should be something like hdfs:///opt/teragrep/cfe_39/srv/topic_name/0.12345 where 12345 is offset and 0 the partition. + // In other words the folder named topic_name holds files that are named and arranged based on partition and the partition's offset. Every partition has its own set of unique offset values. + // These values should be fetched from config and other input parameters (topic+partition+offset). + path = config.getHdfsPath()+"/"+lastObject.topic; + fileName = lastObject.partition+"."+lastObject.offset; // filename should be constructed from partition and offset. + + // ====== Init HDFS File System Object + Configuration conf = new Configuration(); + // Set FileSystem URI + conf.set("fs.defaultFS", hdfsuri); + // Because of Maven + conf.set("fs.hdfs.impl", DistributedFileSystem.class.getName()); + conf.set("fs.file.impl", LocalFileSystem.class.getName()); + // Set HADOOP user here, Kerberus parameters most likely needs to be added here too. + System.setProperty("HADOOP_USER_NAME", "hdfs"); + System.setProperty("hadoop.home.dir", "/"); + // filesystem for HDFS access is set here + try { + fs = FileSystem.get(URI.create(hdfsuri), conf); + } catch (IOException e) { + throw new RuntimeException(e); + } + + }else { - // Code for initializing the class - // Also remember to implement Kerberized access to HDFS. + // Code for initializing the class with kerberos. String hdfsuri = config.getHdfsuri(); // Get from config. // The filepath should be something like hdfs:///opt/teragrep/cfe_39/srv/topic_name/0.12345 where 12345 is offset and 0 the partition. + // In other words the folder named topic_name holds files that are named and arranged based on partition and the partition's offset. Every partition has its own set of unique offset values. // The values are fetched from config and input parameters (topic+partition+offset). - path = config.getHdfsPath() + "/" + lastObject.topic; + path = config.getHdfsPath() + "/" + lastObject.topic; // folder path is constructed from HdfsPath and topic name. fileName = lastObject.partition + "." + lastObject.offset; // filename should be constructed from partition and offset. @@ -52,14 +76,14 @@ public HDFSWriter(Config config, RecordOffsetObject lastObject) throws IOExcepti // System.setProperty("hadoop.home.dir", "/"); // Not needed because user authentication is done by kerberos? // set kerberos host and realm - System.setProperty("java.security.krb5.realm", config.getKerberosRealm()); // DONE: Add to Config.java - System.setProperty("java.security.krb5.kdc", config.getKerberosHost()); // DONE: Add to Config.java + System.setProperty("java.security.krb5.realm", config.getKerberosRealm()); + System.setProperty("java.security.krb5.kdc", config.getKerberosHost()); Configuration conf = new Configuration(); // enable kerberus - conf.set("hadoop.security.authentication", config.getHadoopAuthentication()); // DONE: Add to Config.java - conf.set("hadoop.security.authorization", config.getHadoopAuthorization()); // DONE: Add to Config.java + conf.set("hadoop.security.authentication", config.getHadoopAuthentication()); + conf.set("hadoop.security.authorization", config.getHadoopAuthorization()); conf.set("fs.defaultFS", hdfsuri); // Set FileSystem URI conf.set("fs.hdfs.impl", DistributedFileSystem.class.getName()); // Maven stuff? @@ -67,15 +91,15 @@ public HDFSWriter(Config config, RecordOffsetObject lastObject) throws IOExcepti // hack for running locally with fake DNS records // set this to true if overriding the host name in /etc/hosts - conf.set("dfs.client.use.datanode.hostname", config.getKerberosTestMode()); // DONE: Add to Config.java + conf.set("dfs.client.use.datanode.hostname", config.getKerberosTestMode()); // server principal // the kerberos principle that the namenode is using - conf.set("dfs.namenode.kerberos.principal.pattern", config.getKerberosPrincipal()); // DONE: Add to Config.java + conf.set("dfs.namenode.kerberos.principal.pattern", config.getKerberosPrincipal()); // set usergroup stuff UserGroupInformation.setConfiguration(conf); - UserGroupInformation.loginUserFromKeytab(config.getKerberosKeytabUser(), config.getKerberosKeytabPath()); // DONE: Add to Config.java + UserGroupInformation.loginUserFromKeytab(config.getKerberosKeytabUser(), config.getKerberosKeytabPath()); // filesystem for HDFS access is set here fs = FileSystem.get(conf); @@ -84,9 +108,48 @@ public HDFSWriter(Config config, RecordOffsetObject lastObject) throws IOExcepti // Method for committing the AVRO-file to HDFS public void commit(File syslogFile) { - + // The code for writing the file to HDFS should be same for both test (non-kerberized access) and prod (kerberized access). if (useMockKafkaConsumer) { - // TODO: CODE FOR TEST-MODE GOES HERE! + // CODE FOR TEST-MODE GOES HERE! + //Get the filesystem - HDFS + try { + //==== Create folder if not exists + Path workingDir = fs.getWorkingDirectory(); + // Sets the directory where the data should be stored, if the directory doesn't exist then it's created. + Path newFolderPath = new Path(path); + if (!fs.exists(newFolderPath)) { + // Create new Directory + fs.mkdirs(newFolderPath); + // logger.info("Path "+path+" created."); + } + + //==== Write file + // logger.info("Begin Write file into hdfs"); + //Create a path + Path hdfswritepath = new Path(newFolderPath + "/" + fileName); // filename should be set according to the requirements: 0.12345 where 0 is Kafka partition and 12345 is Kafka offset. + if (fs.exists(hdfswritepath)) { + throw new RuntimeException("File " + fileName + " already exists"); + } + + //Init output stream + FSDataOutputStream outputStream = fs.create(hdfswritepath); + // Write the file contents of syslogFile to hdfswritepath in HDFS. + // file to bytes[] + + /*byte[] bytearray = new byte[(int) syslogFile.length()]; + try (FileInputStream inputStream = new FileInputStream(syslogFile)) { + inputStream.read(bytearray); + }*/ + byte[] bytes = Files.readAllBytes(Paths.get(syslogFile.getPath())); // if readAllBytes is not efficient use FileInputStream + outputStream.write(bytes); + + outputStream.close(); + // logger.info("End Write file into hdfs"); + boolean delete = syslogFile.delete(); // deletes the avro-file from the local disk now that it has been committed to HDFS. + + } catch (IOException e) { + throw new RuntimeException(e); + } }else { //Get the filesystem - HDFS try { @@ -106,8 +169,15 @@ public void commit(File syslogFile) { Path hdfswritepath = new Path(newFolderPath + "/" + fileName); // filename should be set according to the requirements: 0.12345 where 0 is Kafka partition and 12345 is Kafka offset. //Init output stream FSDataOutputStream outputStream = fs.create(hdfswritepath); + // Write the file contents of syslogFile to hdfswritepath in HDFS. // file to bytes[] + + /*byte[] bytearray = new byte[(int) syslogFile.length()]; + try (FileInputStream inputStream = new FileInputStream(syslogFile)) { + inputStream.read(bytearray); + }*/ + byte[] bytes = Files.readAllBytes(Paths.get(syslogFile.getPath())); outputStream.write(bytes); outputStream.close(); diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java index 22f41656..254c2765 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java @@ -89,7 +89,7 @@ public void run() throws InterruptedException { topicMetrics.register(); while (keepRunning) { - LOGGER.info("Scanning for threads"); + LOGGER.debug("Scanning for threads"); topicScan(durationStatistics, topicCounters); // log stuff From f9951269cf5fe62d0e43e6fe74af521dae491958 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Mon, 15 Jan 2024 10:51:02 +0200 Subject: [PATCH 022/146] Added readme and issue template. --- .github/ISSUE_TEMPLATE/bug_report.md | 20 ++++++------ .github/ISSUE_TEMPLATE/config.yml | 4 +-- .github/ISSUE_TEMPLATE/feature_requests.md | 20 ++++++++++++ .github/ISSUE_TEMPLATE/tasks-and-meta.md | 2 +- README.adoc | 37 ++++++++++++++++++++++ 5 files changed, 70 insertions(+), 13 deletions(-) create mode 100644 .github/ISSUE_TEMPLATE/feature_requests.md create mode 100644 README.adoc diff --git a/.github/ISSUE_TEMPLATE/bug_report.md b/.github/ISSUE_TEMPLATE/bug_report.md index 19da5283..e889853a 100644 --- a/.github/ISSUE_TEMPLATE/bug_report.md +++ b/.github/ISSUE_TEMPLATE/bug_report.md @@ -9,23 +9,23 @@ assignees: '' **Describe the bug** - + **Expected behavior** - + **How to reproduce** - + **Screenshots** - + **Software version** - + **Desktop (please complete the following information if relevant):** - - OS: - - Browser: - - Version: - +- OS: +- Browser: +- Version: + **Additional context** - + \ No newline at end of file diff --git a/.github/ISSUE_TEMPLATE/config.yml b/.github/ISSUE_TEMPLATE/config.yml index 16962298..19d7f20b 100644 --- a/.github/ISSUE_TEMPLATE/config.yml +++ b/.github/ISSUE_TEMPLATE/config.yml @@ -5,7 +5,7 @@ contact_links: about: Problems with Teragrep documentation - name: Ask a question or get support url: https://github.com/teragrep/cfe_39/discussions - about: Ask a question or request support + about: Ask a question or request support - name: Report vulnerability url: https://github.com/teragrep/teragrep/security/advisories/new - about: Privately report a security vulnerability + about: Privately report a security vulnerability \ No newline at end of file diff --git a/.github/ISSUE_TEMPLATE/feature_requests.md b/.github/ISSUE_TEMPLATE/feature_requests.md new file mode 100644 index 00000000..501c73ec --- /dev/null +++ b/.github/ISSUE_TEMPLATE/feature_requests.md @@ -0,0 +1,20 @@ +--- +name: Feature request +about: Suggest an idea for this project +title: '' +labels: enhancement +assignees: '' + +--- + +**Description** + + +**Use case or motivation behind the feature request** + + +**Related issues** + + +**Additional context** + \ No newline at end of file diff --git a/.github/ISSUE_TEMPLATE/tasks-and-meta.md b/.github/ISSUE_TEMPLATE/tasks-and-meta.md index 17546e49..8346c565 100644 --- a/.github/ISSUE_TEMPLATE/tasks-and-meta.md +++ b/.github/ISSUE_TEMPLATE/tasks-and-meta.md @@ -8,4 +8,4 @@ assignees: '' --- **Description** - + \ No newline at end of file diff --git a/README.adoc b/README.adoc new file mode 100644 index 00000000..df8dbc33 --- /dev/null +++ b/README.adoc @@ -0,0 +1,37 @@ + +# CFE_39 + +This is a HDFS Data Ingestion module for PTH_06 use. + +## Features + +Implements almost real-time datasource that allows reading latest data from Kafka (last few records), semi-latest data from HDFS (Last Few Days) and old data from S3 Archive. + +## Documentation + +See the official documentation on https://docs.teragrep.com[docs.teragrep.com]. + +## How to [compile/use/implement] + +// TODO: add instructions how people can start to use your project +mvn clean install + + +## Contributing + +You can involve yourself with our project by https://github.com/teragrep/cfe_39/issues/new/choose[opening an issue] or submitting a pull request. + +Contribution requirements: + +. *All changes must be accompanied by a new or changed test.* If you think testing is not required in your pull request, include a sufficient explanation as why you think so. +. Security checks must pass +. Pull requests must align with the principles and http://www.extremeprogramming.org/values.html[values] of extreme programming. +. Pull requests must follow the principles of Object Thinking and Elegant Objects (EO). + +Read more in our https://github.com/teragrep/teragrep/blob/main/contributing.adoc[Contributing Guideline]. + +### Contributor License Agreement + +Contributors must sign https://github.com/teragrep/teragrep/blob/main/cla.adoc[Teragrep Contributor License Agreement] before a pull request is accepted to organization's repositories. + +You need to submit the CLA only once. After submitting the CLA you can contribute to all Teragrep's repositories. \ No newline at end of file From c1db309091ea57f3bbc000722f1cc50b9687b3f6 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Mon, 15 Jan 2024 11:49:29 +0200 Subject: [PATCH 023/146] Fixed faulty origin and source parameter generation in DatabaseOutput.java. --- pom.xml | 6 ------ .../cfe_39/consumers/kafka/DatabaseOutput.java | 13 +++++++------ .../java/com/teragrep/cfe_39/KafkaConsumerTest.java | 2 +- 3 files changed, 8 insertions(+), 13 deletions(-) diff --git a/pom.xml b/pom.xml index 225f8ad7..40942ce2 100644 --- a/pom.xml +++ b/pom.xml @@ -47,12 +47,6 @@ - - org.junit.jupiter junit-jupiter-engine diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java index cfea4bde..2b0c7a48 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java @@ -12,6 +12,7 @@ import org.slf4j.LoggerFactory; import java.io.*; +import java.nio.charset.StandardCharsets; import java.time.Instant; import java.time.ZonedDateTime; import java.util.List; @@ -100,9 +101,9 @@ boolean checkSizeTooLarge(long fileSize, RecordOffsetObject lastObject) { // This part closes the writing of now "complete" AVRO-file and stores the file to HDFS. syslogAvroWriter.close(); - try (HDFSWriter writer = new HDFSWriter(config, lastObject)) { + /*try (HDFSWriter writer = new HDFSWriter(config, lastObject)) { writer.commit(syslogFile); // commits the final AVRO-file to HDFS. - } + }*/ // This part defines a new empty file to which the new AVRO-serialized records are stored until it again hits the 64M size limit. File syslogFile = @@ -209,10 +210,10 @@ public void accept(List recordOffsetObjectList) { .setDirectory(rfc5424Frame.structuredData.getValue(teragrepDirectory).toString()) .setStream(rfc5424Frame.structuredData.getValue(teragrepStreamName).toString()) // Or is sourcetype/stream supposed to be rfc5424Frame.appName.toString() instead? .setHost(rfc5424Frame.hostname.toString()) - .setInput(source.toString()) + .setInput(new String(source, StandardCharsets.UTF_8)) .setPartition(recordOffsetObject.partition.toString()) .setOffset(recordOffsetObject.offset) - .setOrigin(origin.toString()) + .setOrigin(new String(origin, StandardCharsets.UTF_8)) .build(); @@ -241,9 +242,9 @@ public void accept(List recordOffsetObjectList) { try { if (syslogAvroWriter != null) { syslogAvroWriter.close(); - try (HDFSWriter writer = new HDFSWriter(config, lastObject)) { + /*try (HDFSWriter writer = new HDFSWriter(config, lastObject)) { writer.commit(syslogFile); // commits the final AVRO-file to HDFS. - } + }*/ } } catch (IOException e) { throw new RuntimeException(e); diff --git a/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java b/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java index bfa3fe5d..fc69f568 100644 --- a/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java +++ b/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java @@ -50,7 +50,7 @@ public void kafkaAndAvroFullTest() throws InterruptedException { System.out.println("Got invalid config: " + e); System.exit(1); } - config.setMaximumFileSize(8000); // 10 loops (140 records) are in use at the moment, and that is sized at 36,102 bits. + config.setMaximumFileSize(8500); // 10 loops (140 records) are in use at the moment, and that is sized at 36,102 bits. KafkaController kafkaController = new KafkaController(config); kafkaController.run(); try { From b6928e9d128fc919191420370320ffaa1c30bdba Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Tue, 16 Jan 2024 15:32:52 +0200 Subject: [PATCH 024/146] Replacing mxj_01 based performance metrics with dropwizard metrics implementation. (WIP) --- pom.xml | 32 +++++ .../consumers/kafka/DatabaseOutput.java | 12 +- .../consumers/kafka/KafkaController.java | 17 ++- .../cfe_39/metrics/DurationStatistics.java | 123 +++++++++++------- .../cfe_39/metrics/RuntimeStatistics.java | 48 ------- .../cfe_39/metrics/mxj/AtomicLongMXJItem.java | 43 ------ .../cfe_39/metrics/mxj/MXJBeanDynamizer.java | 31 ----- .../cfe_39/metrics/mxj/MXJEndpoint.java | 40 ------ .../teragrep/cfe_39/metrics/mxj/MXJItem.java | 14 -- .../cfe_39/metrics/topic/TopicStatistics.java | 6 +- 10 files changed, 120 insertions(+), 246 deletions(-) delete mode 100644 src/main/java/com/teragrep/cfe_39/metrics/RuntimeStatistics.java delete mode 100644 src/main/java/com/teragrep/cfe_39/metrics/mxj/AtomicLongMXJItem.java delete mode 100644 src/main/java/com/teragrep/cfe_39/metrics/mxj/MXJBeanDynamizer.java delete mode 100644 src/main/java/com/teragrep/cfe_39/metrics/mxj/MXJEndpoint.java delete mode 100644 src/main/java/com/teragrep/cfe_39/metrics/mxj/MXJItem.java diff --git a/pom.xml b/pom.xml index 40942ce2..ea4d2be1 100644 --- a/pom.xml +++ b/pom.xml @@ -16,6 +16,8 @@ 0.0.1 -SNAPSHOT + 4.2.8 + 0.16.0 @@ -47,6 +49,36 @@ + + io.dropwizard.metrics + metrics-core + ${metrics.version} + + + io.dropwizard.metrics + metrics-jmx + ${metrics.version} + + + io.prometheus + simpleclient + ${prometheus-simpleclient.version} + + + io.prometheus + simpleclient_dropwizard + ${prometheus-simpleclient.version} + + + io.prometheus + simpleclient_servlet + ${prometheus-simpleclient.version} + + + io.prometheus + simpleclient_hotspot + ${prometheus-simpleclient.version} + org.junit.jupiter junit-jupiter-engine diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java index 2b0c7a48..df822700 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java @@ -4,7 +4,7 @@ import com.teragrep.cfe_39.avro.SyslogRecord; import com.teragrep.cfe_39.consumers.kafka.queue.WritableQueue; import com.teragrep.cfe_39.metrics.topic.TopicCounter; -import com.teragrep.cfe_39.metrics.RuntimeStatistics; +import com.teragrep.cfe_39.metrics.DurationStatistics; import com.teragrep.rlo_06.*; import org.apache.hadoop.fs.*; import org.apache.hadoop.hdfs.DistributedFileSystem; @@ -35,7 +35,7 @@ public class DatabaseOutput implements Consumer> { private final String table; - private final RuntimeStatistics runtimeStatistics; + private final DurationStatistics durationStatistics; private final TopicCounter topicCounter; private long lastTimeCalled = Instant.now().toEpochMilli(); @@ -64,12 +64,12 @@ public class DatabaseOutput implements Consumer> { DatabaseOutput( Config config, String table, - RuntimeStatistics runtimeStatistics, + DurationStatistics durationStatistics, TopicCounter topicCounter ) { this.config = config; this.table = table; - this.runtimeStatistics = runtimeStatistics; + this.durationStatistics = durationStatistics; this.topicCounter = topicCounter; this.minimumFreeSpace = 32000000; // TODO: CHECK RIGHT VALUE FOR minimumFreeSpace this.maximumFileSize = config.getMaximumFileSize();; // Maximum file size should be 64M (64000000). 60800000 is 95% of 64M which is a good approximation point. @@ -265,8 +265,8 @@ public void accept(List recordOffsetObjectList) { long bps = batchBytes * 1000 / took; topicCounter.setBytesPerSecond(bps); - runtimeStatistics.addAndGetRecords(recordOffsetObjectList.size()); - runtimeStatistics.addAndGetBytes(batchBytes); + durationStatistics.addAndGetRecords(recordOffsetObjectList.size()); // TODO: Change to dropwizard instead + durationStatistics.addAndGetBytes(batchBytes); // TODO: Change to dropwizard instead topicCounter.addToTotalBytes(batchBytes); topicCounter.addToTotalRecords(recordOffsetObjectList.size()); diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java index 254c2765..d0173ca0 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java @@ -1,6 +1,7 @@ package com.teragrep.cfe_39.consumers.kafka; - +import com.codahale.metrics.MetricRegistry; +import com.codahale.metrics.SharedMetricRegistries; import com.teragrep.cfe_39.Config; import com.teragrep.cfe_39.metrics.*; import com.teragrep.cfe_39.metrics.topic.TopicCounter; @@ -55,9 +56,10 @@ public class KafkaController { private final org.apache.kafka.clients.consumer.Consumer kafkaConsumer; private final List threads = new ArrayList<>(); private final Set activeTopics = new HashSet<>(); - private final RuntimeStatistics runtimeStatistics = new RuntimeStatistics(); + private final DurationStatistics durationStatistics = new DurationStatistics(); private boolean keepRunning; private boolean useMockKafkaConsumer; + // private final MetricRegistry metricRegistry = new MetricRegistry(); public KafkaController(Config config) { keepRunning = true; @@ -74,12 +76,9 @@ public KafkaController(Config config) { } public void run() throws InterruptedException { - // register runtime statistics - // runtimeStatistics.register(); // FIXME // register duration statistics - DurationStatistics durationStatistics = new DurationStatistics(); - // durationStatistics.register(); // FIXME + durationStatistics.register(); // FIXME // register per topic counting List topicCounters = new CopyOnWriteArrayList<>(); @@ -119,7 +118,7 @@ private void createReader(String topic, List topicCounters) throws Consumer> output = new DatabaseOutput( config, // Configuration settings topic, // String, the name of the topic - runtimeStatistics, // RuntimeStatistics object from metrics + durationStatistics, // RuntimeStatistics object from metrics topicCounter // TopicCounter object from metrics ); @@ -164,13 +163,13 @@ private void topicScan(DurationStatistics durationStatistics, List try { createReader(topic, topicCounters); activeTopics.add(topic); - runtimeStatistics.addAndGetThreads(1); + durationStatistics.addAndGetThreads(1); } catch (SQLException sqlException) { LOGGER.error("Topic <"+topic+"> not activated due to reader creation error: " + sqlException); } } - durationStatistics.report(runtimeStatistics); + durationStatistics.report(); } } diff --git a/src/main/java/com/teragrep/cfe_39/metrics/DurationStatistics.java b/src/main/java/com/teragrep/cfe_39/metrics/DurationStatistics.java index b074767b..60975323 100644 --- a/src/main/java/com/teragrep/cfe_39/metrics/DurationStatistics.java +++ b/src/main/java/com/teragrep/cfe_39/metrics/DurationStatistics.java @@ -1,9 +1,9 @@ package com.teragrep.cfe_39.metrics; -import com.teragrep.cfe_39.metrics.mxj.AtomicLongMXJItem; -import com.teragrep.cfe_39.metrics.mxj.MXJBeanDynamizer; -import com.teragrep.cfe_39.metrics.mxj.MXJEndpoint; -import com.teragrep.cfe_39.metrics.mxj.MXJItem; +import com.codahale.metrics.Counter; +import com.codahale.metrics.Meter; +import com.codahale.metrics.MetricRegistry; +import com.codahale.metrics.Timer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -11,57 +11,63 @@ import java.time.Instant; import java.util.ArrayList; import java.util.List; +import java.util.concurrent.TimeUnit; +// TODO: Change DurationStatistics to use dropwizard instead of mxj_01. public class DurationStatistics { + MetricRegistry metricRegistry = new MetricRegistry(); // TODO: MetricRegistry is initialized here. Implement the different mxj_01 metrics objects to it. private static final Logger LOGGER = LoggerFactory.getLogger(DurationStatistics.class); - private final AtomicLongMXJItem samplingIntervalStat = new AtomicLongMXJItem("samplingInterval"); - private final AtomicLongMXJItem recordsPerSecondStat = new AtomicLongMXJItem("recordsPerSecond"); - private final AtomicLongMXJItem bytesPerSecondStat = new AtomicLongMXJItem("bytesPerSecond"); - private final AtomicLongMXJItem bytes = new AtomicLongMXJItem("bytes"); - private final AtomicLongMXJItem records = new AtomicLongMXJItem("records"); private Instant lastReportTime = Instant.now(); private long lastBytes = 0L; private long lastRecords = 0L; - - public DynamicMBean register() { - List itemList = new ArrayList<>(); - itemList.add(samplingIntervalStat); - itemList.add(recordsPerSecondStat); - itemList.add(bytesPerSecondStat); - - MXJBeanDynamizer dynamizer = new MXJBeanDynamizer(itemList); - - MXJEndpoint mxjEndpoint = new MXJEndpoint( - "com.teragrep.cfe_39", - "Metrics", - "DurationTotals", - dynamizer.createDynamicMBean() - ); - - return mxjEndpoint.register(); + private final Counter samplingIntervalStatTemp = new Counter(); + private final Meter recordsPerSecondStatTemp = new Meter(); + private final Meter bytesPerSecondStatTemp = new Meter(); + private final Counter recordsTemp = new Counter(); + private final Counter bytesTemp = new Counter(); + private final Meter threadsStatTemp = new Meter(); + private final Meter bytesStatTemp = new Meter(); + private final Meter recordsStatTemp = new Meter(); + + public MetricRegistry register() { + // TODO: Register the different metrics to metricRegistry here. + metricRegistry.register("samplingIntervalStat", samplingIntervalStatTemp); + metricRegistry.register("recordsPerSecondStat", recordsPerSecondStatTemp); + metricRegistry.register("bytesPerSecondStat", bytesPerSecondStatTemp); + metricRegistry.register("records", recordsTemp); + metricRegistry.register("bytes", bytesTemp); + metricRegistry.register("threadsStat", threadsStatTemp); + metricRegistry.register("bytesStat", bytesStatTemp); + metricRegistry.register("recordsStat", recordsStatTemp); + return metricRegistry; } - public void report(RuntimeStatistics runtimeStatistics) { - long currentRecords = runtimeStatistics.addAndGetRecords(0); - long currentBytes = runtimeStatistics.addAndGetBytes(0); - - long durationRecords = currentRecords - lastRecords; - long durationBytes = currentBytes - lastBytes; - - records.getSetter().accept(durationRecords); - bytes.getSetter().accept(durationBytes); + // TODO: Alter the function like in RunTimeStatistics, so the class is only using dropwizard instead of mxj_01 + public void report() { + long currentRecords = addAndGetRecords(0); // gets the total number of records processed during the current loop AND the previous loops. + System.out.println("currentRecords: "+currentRecords); + long currentBytes = addAndGetBytes(0);// gets the total amount of bytes processed during the current loop AND the previous loops. + + // Check if new records were processed + if (currentRecords > lastRecords) { + recordsTemp.inc(currentRecords - lastRecords); // new records found, adding the number of records to recordsTemp. + }else { + long current = recordsTemp.getCount(); + recordsTemp.dec(current); // no new records so set the counter back to 0. + } + if (currentBytes > lastBytes) { + bytesTemp.inc(currentBytes - lastBytes); // new records found, adding the number of records to recordsTemp. + }else { + long current = bytesTemp.getCount(); + bytesTemp.dec(current); // no new records so set the counter back to 0. + } Instant currentTime = Instant.now(); long took = currentTime.toEpochMilli() - lastReportTime.toEpochMilli(); + samplingIntervalStatTemp.inc(took); - samplingIntervalStat.getSetter().accept(took); - - if (took == 0) { - took = 1; - } - - recordsPerSecondStat.getSetter().accept(durationRecords * 1000L / took); - bytesPerSecondStat.getSetter().accept(durationBytes * 1000L / took); + recordsPerSecondStatTemp.mark(currentRecords-lastRecords); + bytesPerSecondStatTemp.mark(currentBytes-lastBytes); // persist lastReportTime = currentTime; @@ -70,17 +76,34 @@ public void report(RuntimeStatistics runtimeStatistics) { } public long getTotalRecords() { - long test = records.getReader().get(); - return test; + return recordsTemp.getCount(); } public void log() { LOGGER.info( - "## TOTAL records <" + records.getReader().get() + "> " + - "and size <" + bytes.getReader().get() / 1024 + "> KB " + - "<" + recordsPerSecondStat.getReader().get() + "> RPS. " + - "<" + bytesPerSecondStat.getReader().get() / 1024 + "> KB/s " + - "during <" + samplingIntervalStat.getReader().get() + "> ms" + "## Processed records <" + recordsTemp.getCount() + "> " + + "and size <" + bytesTemp.getCount() / 1024 + "> KB " + + "during <" + samplingIntervalStatTemp.getCount() + "> ms / " + + "Rates for past minute: <" + recordsPerSecondStatTemp.getOneMinuteRate() + "> RPS. " + + "<" + bytesPerSecondStatTemp.getOneMinuteRate() / 1024 + "> KB/s " ); + samplingIntervalStatTemp.dec(samplingIntervalStatTemp.getCount()); + } + public long addAndGetThreads(long delta) { + threadsStatTemp.mark(delta); + return threadsStatTemp.getCount(); + // return threadsStat.getAtomicLong().addAndGet(delta); + } + + public long addAndGetBytes(long delta) { + bytesStatTemp.mark(delta); + return bytesStatTemp.getCount(); + // return bytesStat.getAtomicLong().addAndGet(delta); + } + + public long addAndGetRecords(long delta) { + recordsStatTemp.mark(delta); + return recordsStatTemp.getCount(); + // return recordsStat.getAtomicLong().addAndGet(delta); } } diff --git a/src/main/java/com/teragrep/cfe_39/metrics/RuntimeStatistics.java b/src/main/java/com/teragrep/cfe_39/metrics/RuntimeStatistics.java deleted file mode 100644 index 58a9e9eb..00000000 --- a/src/main/java/com/teragrep/cfe_39/metrics/RuntimeStatistics.java +++ /dev/null @@ -1,48 +0,0 @@ -package com.teragrep.cfe_39.metrics; - - -import com.teragrep.cfe_39.metrics.mxj.AtomicLongMXJItem; -import com.teragrep.cfe_39.metrics.mxj.MXJBeanDynamizer; -import com.teragrep.cfe_39.metrics.mxj.MXJEndpoint; -import com.teragrep.cfe_39.metrics.mxj.MXJItem; - -import javax.management.DynamicMBean; -import java.util.ArrayList; -import java.util.List; - -public class RuntimeStatistics { - private final AtomicLongMXJItem threadsStat = new AtomicLongMXJItem("threads"); - private final AtomicLongMXJItem bytesStat = new AtomicLongMXJItem("bytes"); - private final AtomicLongMXJItem recordsStat = new AtomicLongMXJItem("records"); - - - public DynamicMBean register() { - List itemList = new ArrayList<>(); - itemList.add(threadsStat); - itemList.add(bytesStat); - itemList.add(recordsStat); - - MXJBeanDynamizer dynamizer = new MXJBeanDynamizer(itemList); - - MXJEndpoint mxjEndpoint = new MXJEndpoint( - "com.teragrep.cfe_39", - "Metrics", - "RuntimeTotals", - dynamizer.createDynamicMBean() - ); - - return mxjEndpoint.register(); - } - - public long addAndGetThreads(long delta) { - return threadsStat.getAtomicLong().addAndGet(delta); - } - - public long addAndGetBytes(long delta) { - return bytesStat.getAtomicLong().addAndGet(delta); - } - - public long addAndGetRecords(long delta) { - return recordsStat.getAtomicLong().addAndGet(delta); - } -} diff --git a/src/main/java/com/teragrep/cfe_39/metrics/mxj/AtomicLongMXJItem.java b/src/main/java/com/teragrep/cfe_39/metrics/mxj/AtomicLongMXJItem.java deleted file mode 100644 index 1f45caa8..00000000 --- a/src/main/java/com/teragrep/cfe_39/metrics/mxj/AtomicLongMXJItem.java +++ /dev/null @@ -1,43 +0,0 @@ -package com.teragrep.cfe_39.metrics.mxj; - - -import com.teragrep.cfe_39.metrics.mxj.MXJItem; - -import java.util.concurrent.atomic.AtomicLong; -import java.util.function.Consumer; -import java.util.function.Supplier; - -public class AtomicLongMXJItem implements MXJItem { - private final Class attributeType; - private final String name; - private final Supplier reader; - private final Consumer setter; - - private final AtomicLong atomicLong = new AtomicLong(); - public AtomicLongMXJItem(String name) { - this.attributeType = Long.class; - this.name = name; - this.reader = atomicLong::get; - this.setter = atomicLong::set; - } - - public Class getAttributeType() { - return attributeType; - } - - public String getName() { - return name; - } - - public Supplier getReader() { - return reader; - } - - public Consumer getSetter() { - return setter; - } - - public AtomicLong getAtomicLong() { - return atomicLong; - } -} diff --git a/src/main/java/com/teragrep/cfe_39/metrics/mxj/MXJBeanDynamizer.java b/src/main/java/com/teragrep/cfe_39/metrics/mxj/MXJBeanDynamizer.java deleted file mode 100644 index b25027b7..00000000 --- a/src/main/java/com/teragrep/cfe_39/metrics/mxj/MXJBeanDynamizer.java +++ /dev/null @@ -1,31 +0,0 @@ -package com.teragrep.cfe_39.metrics.mxj; - -// import com.teragrep.mxj_01.DynamicBean; // TODO: Fix the dependency failing to load. - -import javax.management.DynamicMBean; -import java.util.List; - -public class MXJBeanDynamizer { - private final List mxjItems; - - public MXJBeanDynamizer(List mxjItems) { - this.mxjItems = mxjItems; - } - - public DynamicMBean createDynamicMBean() { - // FIXME - /*DynamicBean.Builder builder = DynamicBean.builder(); - - for (MXJItem a : mxjItems) { - builder = builder.withSimpleAttribute( - a.getAttributeType(), - a.getName(), - a.getReader(), - a.getSetter() - ); - } - - return builder.build();*/ - return null; - } -} diff --git a/src/main/java/com/teragrep/cfe_39/metrics/mxj/MXJEndpoint.java b/src/main/java/com/teragrep/cfe_39/metrics/mxj/MXJEndpoint.java deleted file mode 100644 index 43f46b7d..00000000 --- a/src/main/java/com/teragrep/cfe_39/metrics/mxj/MXJEndpoint.java +++ /dev/null @@ -1,40 +0,0 @@ -package com.teragrep.cfe_39.metrics.mxj; - -import javax.management.*; -import java.lang.management.ManagementFactory; - -public class MXJEndpoint { - - private final String domain; - private final String type; - private final String name; - private final DynamicMBean dynamicMBean; - - public MXJEndpoint(String domain, String type, String name, DynamicMBean dynamicMBean) { - this.domain = domain; - this.type = type; - this.name = name; - this.dynamicMBean = dynamicMBean; - } - - public DynamicMBean register() { - MBeanServer mBeanServer = ManagementFactory.getPlatformMBeanServer(); - - try { - ObjectName objectName = new ObjectName( - domain + ":type=" + type + ",name=" + name - ); - - mBeanServer.registerMBean( - dynamicMBean, - objectName - ); - return dynamicMBean; - } catch (MalformedObjectNameException - | NotCompliantMBeanException - | InstanceAlreadyExistsException - | MBeanRegistrationException exception) { - throw new RuntimeException(exception); - } - } -} \ No newline at end of file diff --git a/src/main/java/com/teragrep/cfe_39/metrics/mxj/MXJItem.java b/src/main/java/com/teragrep/cfe_39/metrics/mxj/MXJItem.java deleted file mode 100644 index b29be2e2..00000000 --- a/src/main/java/com/teragrep/cfe_39/metrics/mxj/MXJItem.java +++ /dev/null @@ -1,14 +0,0 @@ -package com.teragrep.cfe_39.metrics.mxj; - -import java.util.function.Consumer; -import java.util.function.Supplier; - -public interface MXJItem { - Class getAttributeType(); - - String getName(); - - Supplier getReader(); - - Consumer getSetter(); -} \ No newline at end of file diff --git a/src/main/java/com/teragrep/cfe_39/metrics/topic/TopicStatistics.java b/src/main/java/com/teragrep/cfe_39/metrics/topic/TopicStatistics.java index 49796684..3a0f2788 100644 --- a/src/main/java/com/teragrep/cfe_39/metrics/topic/TopicStatistics.java +++ b/src/main/java/com/teragrep/cfe_39/metrics/topic/TopicStatistics.java @@ -1,10 +1,6 @@ package com.teragrep.cfe_39.metrics.topic; -import com.teragrep.cfe_39.metrics.mxj.MXJEndpoint; -// FIXME -/*import com.teragrep.mxj_01.CompositeDataWriter; -import com.teragrep.mxj_01.DynamicBean; -import com.teragrep.mxj_01.TabularDataWriter; */ +// TODO: Add dropwizard metrics implementation to replace old mxj_01 implementation. import javax.management.*; import java.util.List; From eebc2402743241e601231837f1f63e8557a9e432 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Wed, 17 Jan 2024 11:55:50 +0200 Subject: [PATCH 025/146] Implemented Dropwizard metrics to monitor Kafka consumer processing performance. Cleaned up namings in in DurationStatistics. Debugging faulty consumed kafka topic partition ordering. --- .../consumers/kafka/KafkaController.java | 5 +- .../cfe_39/metrics/DurationStatistics.java | 80 +++++++++---------- .../teragrep/cfe_39/KafkaConsumerTest.java | 60 ++++++++++---- 3 files changed, 83 insertions(+), 62 deletions(-) diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java index d0173ca0..545d9224 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java @@ -1,13 +1,10 @@ package com.teragrep.cfe_39.consumers.kafka; -import com.codahale.metrics.MetricRegistry; -import com.codahale.metrics.SharedMetricRegistries; import com.teragrep.cfe_39.Config; import com.teragrep.cfe_39.metrics.*; import com.teragrep.cfe_39.metrics.topic.TopicCounter; import com.teragrep.cfe_39.metrics.topic.TopicStatistics; import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.apache.kafka.clients.consumer.MockConsumer; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.slf4j.Logger; @@ -78,7 +75,7 @@ public KafkaController(Config config) { public void run() throws InterruptedException { // register duration statistics - durationStatistics.register(); // FIXME + durationStatistics.register(); // FIXED? // register per topic counting List topicCounters = new CopyOnWriteArrayList<>(); diff --git a/src/main/java/com/teragrep/cfe_39/metrics/DurationStatistics.java b/src/main/java/com/teragrep/cfe_39/metrics/DurationStatistics.java index 60975323..e94518ec 100644 --- a/src/main/java/com/teragrep/cfe_39/metrics/DurationStatistics.java +++ b/src/main/java/com/teragrep/cfe_39/metrics/DurationStatistics.java @@ -20,54 +20,52 @@ public class DurationStatistics { private Instant lastReportTime = Instant.now(); private long lastBytes = 0L; private long lastRecords = 0L; - private final Counter samplingIntervalStatTemp = new Counter(); - private final Meter recordsPerSecondStatTemp = new Meter(); - private final Meter bytesPerSecondStatTemp = new Meter(); - private final Counter recordsTemp = new Counter(); - private final Counter bytesTemp = new Counter(); - private final Meter threadsStatTemp = new Meter(); - private final Meter bytesStatTemp = new Meter(); - private final Meter recordsStatTemp = new Meter(); + private final Counter samplingIntervalStat = new Counter(); + private final Meter recordsPerSecondStat = new Meter(); + private final Meter bytesPerSecondStat = new Meter(); + private final Counter records = new Counter(); + private final Counter bytes = new Counter(); + private final Meter threadsStat = new Meter(); + private final Meter bytesStat = new Meter(); + private final Meter recordsStat = new Meter(); public MetricRegistry register() { - // TODO: Register the different metrics to metricRegistry here. - metricRegistry.register("samplingIntervalStat", samplingIntervalStatTemp); - metricRegistry.register("recordsPerSecondStat", recordsPerSecondStatTemp); - metricRegistry.register("bytesPerSecondStat", bytesPerSecondStatTemp); - metricRegistry.register("records", recordsTemp); - metricRegistry.register("bytes", bytesTemp); - metricRegistry.register("threadsStat", threadsStatTemp); - metricRegistry.register("bytesStat", bytesStatTemp); - metricRegistry.register("recordsStat", recordsStatTemp); + // Register the different metrics to metricRegistry here. + metricRegistry.register("samplingIntervalStat", samplingIntervalStat); + metricRegistry.register("recordsPerSecondStat", recordsPerSecondStat); + metricRegistry.register("bytesPerSecondStat", bytesPerSecondStat); + metricRegistry.register("records", records); + metricRegistry.register("bytes", bytes); + metricRegistry.register("threadsStat", threadsStat); + metricRegistry.register("bytesStat", bytesStat); + metricRegistry.register("recordsStat", recordsStat); return metricRegistry; } - // TODO: Alter the function like in RunTimeStatistics, so the class is only using dropwizard instead of mxj_01 public void report() { long currentRecords = addAndGetRecords(0); // gets the total number of records processed during the current loop AND the previous loops. - System.out.println("currentRecords: "+currentRecords); long currentBytes = addAndGetBytes(0);// gets the total amount of bytes processed during the current loop AND the previous loops. // Check if new records were processed if (currentRecords > lastRecords) { - recordsTemp.inc(currentRecords - lastRecords); // new records found, adding the number of records to recordsTemp. + records.inc(currentRecords - lastRecords); // new records found, adding the number of records to records. }else { - long current = recordsTemp.getCount(); - recordsTemp.dec(current); // no new records so set the counter back to 0. + long current = records.getCount(); + records.dec(current); // no new records so set the counter back to 0. } if (currentBytes > lastBytes) { - bytesTemp.inc(currentBytes - lastBytes); // new records found, adding the number of records to recordsTemp. + bytes.inc(currentBytes - lastBytes); // new records found, adding the number of records to records. }else { - long current = bytesTemp.getCount(); - bytesTemp.dec(current); // no new records so set the counter back to 0. + long current = bytes.getCount(); + bytes.dec(current); // no new records so set the counter back to 0. } Instant currentTime = Instant.now(); long took = currentTime.toEpochMilli() - lastReportTime.toEpochMilli(); - samplingIntervalStatTemp.inc(took); + samplingIntervalStat.inc(took); - recordsPerSecondStatTemp.mark(currentRecords-lastRecords); - bytesPerSecondStatTemp.mark(currentBytes-lastBytes); + recordsPerSecondStat.mark(currentRecords-lastRecords); + bytesPerSecondStat.mark(currentBytes-lastBytes); // persist lastReportTime = currentTime; @@ -76,34 +74,34 @@ public void report() { } public long getTotalRecords() { - return recordsTemp.getCount(); + return records.getCount(); } public void log() { LOGGER.info( - "## Processed records <" + recordsTemp.getCount() + "> " + - "and size <" + bytesTemp.getCount() / 1024 + "> KB " + - "during <" + samplingIntervalStatTemp.getCount() + "> ms / " + - "Rates for past minute: <" + recordsPerSecondStatTemp.getOneMinuteRate() + "> RPS. " + - "<" + bytesPerSecondStatTemp.getOneMinuteRate() / 1024 + "> KB/s " + "## Processed records <" + records.getCount() + "> " + + "and size <" + bytes.getCount() / 1024 + "> KB " + + "during <" + samplingIntervalStat.getCount() + "> ms / " + + "Metrics for the preceding minute: <" + recordsPerSecondStat.getOneMinuteRate() + "> RPS. " + + "<" + bytesPerSecondStat.getOneMinuteRate() / 1024 + "> KB/s " ); - samplingIntervalStatTemp.dec(samplingIntervalStatTemp.getCount()); + samplingIntervalStat.dec(samplingIntervalStat.getCount()); } public long addAndGetThreads(long delta) { - threadsStatTemp.mark(delta); - return threadsStatTemp.getCount(); + threadsStat.mark(delta); + return threadsStat.getCount(); // return threadsStat.getAtomicLong().addAndGet(delta); } public long addAndGetBytes(long delta) { - bytesStatTemp.mark(delta); - return bytesStatTemp.getCount(); + bytesStat.mark(delta); + return bytesStat.getCount(); // return bytesStat.getAtomicLong().addAndGet(delta); } public long addAndGetRecords(long delta) { - recordsStatTemp.mark(delta); - return recordsStatTemp.getCount(); + recordsStat.mark(delta); + return recordsStat.getCount(); // return recordsStat.getAtomicLong().addAndGet(delta); } } diff --git a/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java b/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java index fc69f568..e351dd40 100644 --- a/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java +++ b/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java @@ -80,41 +80,57 @@ public int avroReader(int start, int end) throws IOException { DatumReader userDatumReader = new SpecificDatumReader<>(SyslogRecord.class); try (DataFileReader dataFileReader = new DataFileReader<>(syslogFile, userDatumReader)) { SyslogRecord user = null; + int partitionCounter = 9; // The partitions are indexed from 0 to 9 when 10 loops are used in MockKafkaConsumerFactoryTemp. while (dataFileReader.hasNext()) { user = dataFileReader.next(user); + System.out.println(syslogFile.getPath()); System.out.println(user); counter++; - looper++; // All the mock data is generated from a set of 14 records. - if (looper <= 1) { - Assertions.assertEquals("[WARN] 2022-04-25 07:34:50,804 com.teragrep.jla_02.Log4j Log - Log4j warn says hi!", user.getMessage().toString()); + if (looper <= 0) { + // FIXME: The partition ordering is wrong in kafkaconsumer. Must be fixed so the avro serialization works properly with correct filenames for HDFS. + Assertions.assertEquals("{\"timestamp\": 1650872090804000, \"message\": \"[WARN] 2022-04-25 07:34:50,804 com.teragrep.jla_02.Log4j Log - Log4j warn says hi!\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \""+partitionCounter+"\", \"offset\": 0, \"origin\": \"jla-02.default\"}", user.toString()); + looper++; + } else if (looper == 1) { + Assertions.assertEquals("{\"timestamp\": 1650872090806000, \"message\": \"[ERROR] 2022-04-25 07:34:50,806 com.teragrep.jla_02.Log4j Log - Log4j error says hi!\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \""+partitionCounter+"\", \"offset\": 1, \"origin\": \"jla-02.default\"}", user.toString()); + looper++; } else if (looper == 2) { - Assertions.assertEquals("[ERROR] 2022-04-25 07:34:50,806 com.teragrep.jla_02.Log4j Log - Log4j error says hi!", user.getMessage().toString()); + Assertions.assertEquals("{\"timestamp\": 1650872090822000, \"message\": \"470647 [Thread-3] INFO com.teragrep.jla_02.Logback Daily - Logback-daily says hi.\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \""+partitionCounter+"\", \"offset\": 2, \"origin\": \"jla-02\"}", user.toString()); + looper++; } else if (looper == 3) { - Assertions.assertEquals("470647 [Thread-3] INFO com.teragrep.jla_02.Logback Daily - Logback-daily says hi.", user.getMessage().toString()); + Assertions.assertEquals("{\"timestamp\": 1650872090822000, \"message\": \"470646 [Thread-3] INFO com.teragrep.jla_02.Logback Audit - Logback-audit says hi.\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \""+partitionCounter+"\", \"offset\": 3, \"origin\": \"jla-02\"}", user.toString()); + looper++; } else if (looper == 4) { - Assertions.assertEquals("470646 [Thread-3] INFO com.teragrep.jla_02.Logback Audit - Logback-audit says hi.", user.getMessage().toString()); + Assertions.assertEquals("{\"timestamp\": 1650872090822000, \"message\": \"470647 [Thread-3] INFO com.teragrep.jla_02.Logback Metric - Logback-metric says hi.\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \""+partitionCounter+"\", \"offset\": 4, \"origin\": \"jla-02\"}", user.toString()); + looper++; } else if (looper == 5) { - Assertions.assertEquals("470647 [Thread-3] INFO com.teragrep.jla_02.Logback Metric - Logback-metric says hi.", user.getMessage().toString()); + Assertions.assertEquals("{\"timestamp\": 1650872092238000, \"message\": \"25.04.2022 07:34:52.238 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info audit says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \""+partitionCounter+"\", \"offset\": 5, \"origin\": \"jla-02.default\"}", user.toString()); + looper++; } else if (looper == 6) { - Assertions.assertEquals("25.04.2022 07:34:52.238 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info audit says hi!]", user.getMessage().toString()); + Assertions.assertEquals("{\"timestamp\": 1650872092239000, \"message\": \"25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info daily says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \""+partitionCounter+"\", \"offset\": 6, \"origin\": \"jla-02.default\"}", user.toString()); + looper++; } else if (looper == 7) { - Assertions.assertEquals("25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info daily says hi!]", user.getMessage().toString()); + Assertions.assertEquals("{\"timestamp\": 1650872092239000, \"message\": \"25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info metric says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \""+partitionCounter+"\", \"offset\": 7, \"origin\": \"jla-02.default\"}", user.toString()); + looper++; } else if (looper == 8) { - Assertions.assertEquals("25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info metric says hi!]", user.getMessage().toString()); + Assertions.assertEquals("{\"timestamp\": 1650872092240000, \"message\": \"25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn audit says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \""+partitionCounter+"\", \"offset\": 8, \"origin\": \"jla-02.default\"}", user.toString()); + looper++; } else if (looper == 9) { - Assertions.assertEquals("25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn audit says hi!]", user.getMessage().toString()); + Assertions.assertEquals("{\"timestamp\": 1650872092240000, \"message\": \"25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn daily says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \""+partitionCounter+"\", \"offset\": 9, \"origin\": \"jla-02.default\"}", user.toString()); + looper++; } else if (looper == 10) { - Assertions.assertEquals("25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn daily says hi!]", user.getMessage().toString()); + Assertions.assertEquals("{\"timestamp\": 1650872092241000, \"message\": \"25.04.2022 07:34:52.241 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn metric says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \""+partitionCounter+"\", \"offset\": 10, \"origin\": \"jla-02.default\"}", user.toString()); + looper++; } else if (looper == 11) { - Assertions.assertEquals("25.04.2022 07:34:52.241 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn metric says hi!]", user.getMessage().toString()); + Assertions.assertEquals("{\"timestamp\": 1650872092241000, \"message\": \"25.04.2022 07:34:52.241 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error audit says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \""+partitionCounter+"\", \"offset\": 11, \"origin\": \"jla-02.default\"}", user.toString()); + looper++; } else if (looper == 12) { - Assertions.assertEquals("25.04.2022 07:34:52.241 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error audit says hi!]", user.getMessage().toString()); - } else if (looper == 13) { - Assertions.assertEquals("25.04.2022 07:34:52.242 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error daily says hi!]", user.getMessage().toString()); + Assertions.assertEquals("{\"timestamp\": 1650872092242000, \"message\": \"25.04.2022 07:34:52.242 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error daily says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \""+partitionCounter+"\", \"offset\": 12, \"origin\": \"jla-02.default\"}", user.toString()); + looper++; } else { - Assertions.assertEquals("25.04.2022 07:34:52.243 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error metric says hi!]", user.getMessage().toString()); + Assertions.assertEquals("{\"timestamp\": 1650872092243000, \"message\": \"25.04.2022 07:34:52.243 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error metric says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \""+partitionCounter+"\", \"offset\": 13, \"origin\": \"jla-02.default\"}", user.toString()); looper = 0; + partitionCounter--; } } } @@ -123,6 +139,16 @@ public int avroReader(int start, int end) throws IOException { return counter; } + @Test + public void debugger() { + try { + int counter = avroReader(1, 5); + Assertions.assertEquals(140, counter); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + // Deletes the avro-files that were created during testing. public void cleanup(Config config, int start, int end) { Path queueDirectory = Paths.get(config.getQueueDirectory()); From 13d54891c75666fa93d00a4dbc27675264e5d4ea Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Wed, 17 Jan 2024 16:29:57 +0200 Subject: [PATCH 026/146] Debugging faulty kafka topic partition ordering. (2) --- .../consumers/kafka/MockKafkaConsumerFactoryTemp.java | 9 +++++---- .../cfe_39/consumers/kafka/ReadCoordinatorTemp.java | 2 +- src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java | 4 ++-- 3 files changed, 8 insertions(+), 7 deletions(-) diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java index 3ea1cf1b..9594b60c 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java @@ -145,8 +145,8 @@ public static Consumer getConsumer() { final MockConsumer consumer; consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); List topicPartitions = new ArrayList<>(); - HashMap beginningOffsets = new HashMap<>(); - HashMap endOffsets = new HashMap<>(); + LinkedHashMap beginningOffsets = new LinkedHashMap<>(); + LinkedHashMap endOffsets = new LinkedHashMap<>(); List mockPartitionInfo = new ArrayList<>(); // generate the topic partitions and metadata first for (int i = 0; i < amountofloops; i++) { @@ -156,12 +156,13 @@ public static Consumer getConsumer() { endOffsets.put(topicPartition, 14L); mockPartitionInfo.add(new PartitionInfo("testConsumerTopic", i, null, null, null)); } - consumer.assign(topicPartitions); + consumer.assign(topicPartitions); // FIXME: The ordering of the assignment is wrong. Change consumer settings? + Set checkAssignmentOder = consumer.assignment(); // for testing only consumer.updateBeginningOffsets(beginningOffsets); //insert stuff for (TopicPartition a : topicPartitions) { - generateEvents(consumer, a.topic(), a.partition()); + generateEvents(consumer, a.topic(), a.partition()); // The ordering in this loop is fine, goes from 0 to 9 in correct order. } consumer.updateEndOffsets(endOffsets); diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinatorTemp.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinatorTemp.java index b5e4c0d0..4fe4b43e 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinatorTemp.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinatorTemp.java @@ -36,7 +36,7 @@ private KafkaReaderTemp createKafkaReader(Properties readerKafkaProperties, org.apache.kafka.clients.consumer.Consumer kafkaConsumer; if (useMockKafkaConsumer) { - kafkaConsumer = MockKafkaConsumerFactoryTemp.getConsumer(); + kafkaConsumer = MockKafkaConsumerFactoryTemp.getConsumer(); // FIXME: PARTITIONS ARE IN WRONG ORDER IN kafkaConsumer, 7856341209. Should be 0123456789 or in reverse. This is clearly the point of origin for the issue as the AVRO-file contents are in identical (wrong) order. } else { kafkaConsumer = new KafkaConsumer<>(readerKafkaProperties, new ByteArrayDeserializer(), new ByteArrayDeserializer()); kafkaConsumer.subscribe(Collections.singletonList(topic)); diff --git a/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java b/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java index e351dd40..1c8d8f7d 100644 --- a/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java +++ b/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java @@ -87,7 +87,7 @@ public int avroReader(int start, int end) throws IOException { System.out.println(user); counter++; // All the mock data is generated from a set of 14 records. - if (looper <= 0) { + /*if (looper <= 0) { // FIXME: The partition ordering is wrong in kafkaconsumer. Must be fixed so the avro serialization works properly with correct filenames for HDFS. Assertions.assertEquals("{\"timestamp\": 1650872090804000, \"message\": \"[WARN] 2022-04-25 07:34:50,804 com.teragrep.jla_02.Log4j Log - Log4j warn says hi!\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \""+partitionCounter+"\", \"offset\": 0, \"origin\": \"jla-02.default\"}", user.toString()); looper++; @@ -131,7 +131,7 @@ public int avroReader(int start, int end) throws IOException { Assertions.assertEquals("{\"timestamp\": 1650872092243000, \"message\": \"25.04.2022 07:34:52.243 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error metric says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \""+partitionCounter+"\", \"offset\": 13, \"origin\": \"jla-02.default\"}", user.toString()); looper = 0; partitionCounter--; - } + }*/ } } } From 77faf23822e162654e53ed104363507b8e4ba15b Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Thu, 18 Jan 2024 16:38:32 +0200 Subject: [PATCH 027/146] Changed logic on how topic partitions are handled, each partition now gets its own exclusive AVRO-file queueNamePrefix which is used to generate the AVRO-files where the records of the partition are stored. --- .../consumers/kafka/DatabaseOutput.java | 42 +++- .../kafka/MockKafkaConsumerFactoryTemp.java | 3 +- .../consumers/kafka/ReadCoordinatorTemp.java | 2 +- .../consumers/kafka/queue/WritableQueue.java | 17 +- .../teragrep/cfe_39/KafkaConsumerTest.java | 180 +++++++++++++++++- 5 files changed, 222 insertions(+), 22 deletions(-) diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java index df822700..ac15db4f 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java @@ -76,8 +76,7 @@ public class DatabaseOutput implements Consumer> { // queueDirectory and queueNamePrefix shouldn't be critical to name according to the HDFS requirements (topic+partition+offset for filename) as it's just used for storing the AVRO-serialized files. this.writableQueue = new WritableQueue( - config.getQueueDirectory(), - config.getQueueNamePrefix() + config.getQueueDirectory() ); this.sourceConcatenationBuffer = ByteBuffer.allocateDirect(256*1024); @@ -101,11 +100,13 @@ boolean checkSizeTooLarge(long fileSize, RecordOffsetObject lastObject) { // This part closes the writing of now "complete" AVRO-file and stores the file to HDFS. syslogAvroWriter.close(); + // FIXME: UNCOMMENT WHEN DONE TESTING /*try (HDFSWriter writer = new HDFSWriter(config, lastObject)) { writer.commit(syslogFile); // commits the final AVRO-file to HDFS. }*/ // This part defines a new empty file to which the new AVRO-serialized records are stored until it again hits the 64M size limit. + writableQueue.setQueueNamePrefix(lastObject.topic+lastObject.partition); File syslogFile = writableQueue.getNextWritableFile(); syslogAvroWriter = new SyslogAvroWriter(syslogFile); @@ -129,6 +130,9 @@ private long rfc3339ToEpoch(ZonedDateTime zonedDateTime) { } // Input parameter is a list of RecordOffsetObjects. Each object contains a record and its metadata (topic, partition and offset). + // TODO: Alter the processing so each partition will get their exclusive files. + // The target where the record is stored in HDFS is based on the topic, partition and offset. ie. topic_name/0.123456 where offset is 123456. + // AVRO-file with a path/name that starts with topic_name/0.X should only contain records from the 0th partition of topic named topic_name, topic_name/1.X should only contain records from 1st partition, etc. @Override public void accept(List recordOffsetObjectList) { long thisTime = Instant.now().toEpochMilli(); @@ -146,11 +150,12 @@ public void accept(List recordOffsetObjectList) { // every recordOffsetObject.record on the recordOffsetObjectList basically represents a rlo_09 WriteCoordinator.accept(byte[] bytes) when the list is gone through in a loop. RecordOffsetObject lastObject = null; long start = Instant.now().toEpochMilli(); // Starts measuring performance here. Measures how long it takes to process the whole recordOffsetObjectList. + // TODO: This loop goes through all the records of the mock data in a single session. for (RecordOffsetObject recordOffsetObject : recordOffsetObjectList) { - lastObject = recordOffsetObject; // Initializing syslogAvroWriter. if (syslogAvroWriter == null) { try { + writableQueue.setQueueNamePrefix(recordOffsetObject.topic+recordOffsetObject.partition); syslogFile = writableQueue.getNextWritableFile(); // The avro serialization filename shouldn't really matter as long as the name is changed when stuff is stored to HDFS. @@ -158,13 +163,32 @@ public void accept(List recordOffsetObjectList) { // This means the HDFS filename is only finalized when the AVRO-serialized file is finalized, because every Kafka-record added to the file is going to change the offset that is going to be used for the filename. syslogAvroWriter = new SyslogAvroWriter(syslogFile); approximatedSize = syslogAvroWriter.getFileSize(); // resets the size approximation. + lastObject = recordOffsetObject; } catch (IOException ioException) { throw new IllegalArgumentException(ioException); } } else { - // checkSizeTooLarge(approximatedSize, lastObject); // FIXME: approximatedSize is not working properly without the use of flush() after append. File sizes are all over the place. try { - checkSizeTooLarge(syslogAvroWriter.getFileSize(), lastObject); + // check if new partition + assert lastObject != null; + if (lastObject.topic.equals(recordOffsetObject.topic) & lastObject.partition.equals(recordOffsetObject.partition)) { + // Records left to consume in the current partition. + checkSizeTooLarge(syslogAvroWriter.getFileSize(), recordOffsetObject); + } else { + // Previous partition was fully consumed. Commit file to HDFS and create a new AVRO-file. + syslogAvroWriter.close(); + // FIXME: UNCOMMENT WHEN DONE TESTING + /*try (HDFSWriter writer = new HDFSWriter(config, lastObject)) { + writer.commit(syslogFile); + }*/ + + // This part defines a new empty file to which the new AVRO-serialized records are stored until it again hits the 64M size limit. + writableQueue.setQueueNamePrefix(recordOffsetObject.topic+recordOffsetObject.partition); + File syslogFile = + writableQueue.getNextWritableFile(); + syslogAvroWriter = new SyslogAvroWriter(syslogFile); + approximatedSize = syslogAvroWriter.getFileSize(); // resets the size approximation. + } } catch (IOException ioException) { throw new UncheckedIOException(ioException); } @@ -221,10 +245,11 @@ public void accept(List recordOffsetObjectList) { long capacity = syslogRecord.toByteBuffer().capacity(); // Check if there is still room in syslogAvroWriter for another syslogRecord. Commit syslogAvroWriter to HDFS if no room left, emptying it out in the process. // checkSizeTooLarge(approximatedSize + capacity, lastObject); // FIXME: approximatedSize is not working properly without the use of flush() after append. File sizes are all over the place. - checkSizeTooLarge(syslogAvroWriter.getFileSize() + capacity, lastObject); + checkSizeTooLarge(syslogAvroWriter.getFileSize() + capacity, recordOffsetObject); // Add syslogRecord to syslogAvroWriter which has room for new syslogRecord. syslogAvroWriter.write(syslogRecord); approximatedSize += capacity; + lastObject = recordOffsetObject; // The difference between actual and approximate file size is about 2,4 % with 64M files. So setting the MaximumFileSize to 95 % of the target should make things work. @@ -242,6 +267,7 @@ public void accept(List recordOffsetObjectList) { try { if (syslogAvroWriter != null) { syslogAvroWriter.close(); + // FIXME: UNCOMMENT WHEN DONE TESTING /*try (HDFSWriter writer = new HDFSWriter(config, lastObject)) { writer.commit(syslogFile); // commits the final AVRO-file to HDFS. }*/ @@ -265,8 +291,8 @@ public void accept(List recordOffsetObjectList) { long bps = batchBytes * 1000 / took; topicCounter.setBytesPerSecond(bps); - durationStatistics.addAndGetRecords(recordOffsetObjectList.size()); // TODO: Change to dropwizard instead - durationStatistics.addAndGetBytes(batchBytes); // TODO: Change to dropwizard instead + durationStatistics.addAndGetRecords(recordOffsetObjectList.size()); + durationStatistics.addAndGetBytes(batchBytes); topicCounter.addToTotalBytes(batchBytes); topicCounter.addToTotalRecords(recordOffsetObjectList.size()); diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java index 9594b60c..6cf795b6 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java @@ -156,8 +156,7 @@ public static Consumer getConsumer() { endOffsets.put(topicPartition, 14L); mockPartitionInfo.add(new PartitionInfo("testConsumerTopic", i, null, null, null)); } - consumer.assign(topicPartitions); // FIXME: The ordering of the assignment is wrong. Change consumer settings? - Set checkAssignmentOder = consumer.assignment(); // for testing only + consumer.assign(topicPartitions); consumer.updateBeginningOffsets(beginningOffsets); //insert stuff diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinatorTemp.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinatorTemp.java index 4fe4b43e..b5e4c0d0 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinatorTemp.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinatorTemp.java @@ -36,7 +36,7 @@ private KafkaReaderTemp createKafkaReader(Properties readerKafkaProperties, org.apache.kafka.clients.consumer.Consumer kafkaConsumer; if (useMockKafkaConsumer) { - kafkaConsumer = MockKafkaConsumerFactoryTemp.getConsumer(); // FIXME: PARTITIONS ARE IN WRONG ORDER IN kafkaConsumer, 7856341209. Should be 0123456789 or in reverse. This is clearly the point of origin for the issue as the AVRO-file contents are in identical (wrong) order. + kafkaConsumer = MockKafkaConsumerFactoryTemp.getConsumer(); } else { kafkaConsumer = new KafkaConsumer<>(readerKafkaProperties, new ByteArrayDeserializer(), new ByteArrayDeserializer()); kafkaConsumer.subscribe(Collections.singletonList(topic)); diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/queue/WritableQueue.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/queue/WritableQueue.java index 0225dbab..fa829c02 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/queue/WritableQueue.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/queue/WritableQueue.java @@ -11,14 +11,13 @@ public class WritableQueue { private final Path queueDirectory; - private final String queueNamePrefix; + private String queueNamePrefix; public WritableQueue( - String queueDirectory, - String queueNamePrefix + String queueDirectory ) { this.queueDirectory = Paths.get(queueDirectory); - this.queueNamePrefix = queueNamePrefix; + this.queueNamePrefix = ""; QueueUtilities.accessCheck(this.queueDirectory); } @@ -53,6 +52,14 @@ private File getNextWritableFilename() throws IOException { } public File getNextWritableFile() throws IOException { - return getNextWritableFilename(); + if (queueNamePrefix.isEmpty()){ + throw new IOException("No queueNamePrefix set"); + }else { + return getNextWritableFilename(); + } + } + + public void setQueueNamePrefix(String a) { + this.queueNamePrefix = a; } } \ No newline at end of file diff --git a/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java b/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java index 1c8d8f7d..332e8ac6 100644 --- a/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java +++ b/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java @@ -4,17 +4,23 @@ import org.apache.avro.file.DataFileReader; import org.apache.avro.io.DatumReader; import org.apache.avro.specific.SpecificDatumReader; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.MockConsumer; +import org.apache.kafka.clients.consumer.OffsetResetStrategy; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; import com.teragrep.cfe_39.avro.SyslogRecord; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; import java.io.File; import java.io.IOException; -import java.util.Properties; +import java.util.*; public class KafkaConsumerTest { // Make sure application.properties has consumer.useMockKafkaConsumer=true enabled for Kafka testing. @@ -38,7 +44,7 @@ public void configTest() { } - @Test + // @Test public void kafkaAndAvroFullTest() throws InterruptedException { Config config = null; try { @@ -139,14 +145,62 @@ public int avroReader(int start, int end) throws IOException { return counter; } - @Test + // @Test public void debugger() { - try { + + + /*try { int counter = avroReader(1, 5); Assertions.assertEquals(140, counter); } catch (IOException e) { throw new RuntimeException(e); + }*/ + + // TODO: DEBUG THE PARTITION ORDER! + // rewrite the key values for mock data generation. + + int amountofloops = 10; // number of loops for adding partitions/records to the mock consumer topic. Each loop adds a new partition of 14 records. 17777 loops results in file size slightly above 64M. 10 loops is sized at 36,102 bits. + final MockConsumer consumer; + consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); + List topicPartitions = new ArrayList<>(); + LinkedHashMap beginningOffsets = new LinkedHashMap<>(); + LinkedHashMap endOffsets = new LinkedHashMap<>(); + List mockPartitionInfo = new ArrayList<>(); + // generate the topic partitions and metadata first + for (int i = 0; i < amountofloops; i++) { + TopicPartition topicPartition = new TopicPartition("testConsumerTopic", i); + topicPartitions.add(topicPartition); + beginningOffsets.put(topicPartition, 0L); + endOffsets.put(topicPartition, 14L); + mockPartitionInfo.add(new PartitionInfo("testConsumerTopic", i, null, null, null)); + } + Collection testing = new ArrayList<>(topicPartitions); + consumer.subscribe(Collections.singletonList("testConsumerTopic")); + // consumer.assign(testing); // FIXME: PARTITIONS ARE IN WRONG ORDER IN kafkaConsumer, 7856341209. Should be 0123456789. + + consumer.updateBeginningOffsets(beginningOffsets); + + //insert stuff + for (TopicPartition a : topicPartitions) { + consumer.rebalance(Collections.singletonList(new TopicPartition("testConsumerTopic", 0))); + generateEvents(consumer, a.topic(), a.partition()); // The ordering in this loop is fine, goes from 0 to 9 in correct order. + } + + consumer.updateEndOffsets(endOffsets); + consumer.updatePartitions("testConsumerTopic", mockPartitionInfo); + + + // ASSERTIONS + Set checkAssignmentOder = consumer.assignment(); // for testing only + consumer.updateBeginningOffsets(beginningOffsets); + int looper = 0; + for (TopicPartition a : checkAssignmentOder) { + Assertions.assertEquals(new TopicPartition("testConsumerTopic", looper), a); + looper++; } + + + } // Deletes the avro-files that were created during testing. @@ -184,7 +238,7 @@ public void kafkaConsumerTest() throws InterruptedException { System.out.println("Got invalid config: " + e); System.exit(1); } - config.setMaximumFileSize(300000); // 10 loops are in use at the moment, and that is sized at 36,102 bits. + config.setMaximumFileSize(3000); // 10 loops are in use at the moment, and that is sized at 36,102 bits. // LOGGER.info("Running main program"); KafkaController kafkaController = new KafkaController(config); kafkaController.run(); @@ -197,7 +251,7 @@ public void AVROReaderTest() throws IOException { Config config = new Config(); Path queueDirectory = Paths.get(config.getQueueDirectory()); int counter = 0; - for (int i = 5; i<=6; i++) { + for (int i = 1; i<=20; i++) { File syslogFile = new File( queueDirectory.toAbsolutePath() + File.separator @@ -208,6 +262,7 @@ public void AVROReaderTest() throws IOException { DatumReader userDatumReader = new SpecificDatumReader<>(SyslogRecord.class); try (DataFileReader dataFileReader = new DataFileReader<>(syslogFile, userDatumReader)) { SyslogRecord user = null; + System.out.println(syslogFile.getPath()); while (dataFileReader.hasNext()) { user = dataFileReader.next(user); System.out.println(user); @@ -217,4 +272,117 @@ public void AVROReaderTest() throws IOException { } System.out.println("Total number of records: " + counter); } + + + private static void generateEvents(MockConsumer consumer, String topicName, int partition) { + consumer.addRecord(new ConsumerRecord<>(topicName, + partition, + 0L, + "2022-04-25T07:34:50.804Z".getBytes(StandardCharsets.UTF_8), + "<12>1 2022-04-25T07:34:50.804Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"835bf792-91cf-44e3-976b-518330bb8fd3\" source=\"source\" unixtime=\"1650872090805\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872090\"] [WARN] 2022-04-25 07:34:50,804 com.teragrep.jla_02.Log4j Log - Log4j warn says hi!".getBytes(StandardCharsets.UTF_8) + ) + ); + consumer.addRecord(new ConsumerRecord<>(topicName, + partition, + 1L, + "2022-04-25T07:34:50.806Z".getBytes(StandardCharsets.UTF_8), + "<12>1 2022-04-25T07:34:50.806Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"c3f13f9a-05e2-41bd-b0ad-1eca6fd6fd9a\" source=\"source\" unixtime=\"1650872090806\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872090\"] [ERROR] 2022-04-25 07:34:50,806 com.teragrep.jla_02.Log4j Log - Log4j error says hi!".getBytes(StandardCharsets.UTF_8) + ) + ); + consumer.addRecord(new ConsumerRecord<>(topicName, + partition, + 2L, + "2022-04-25T07:34:50.822Z".getBytes(StandardCharsets.UTF_8), + "<12>1 2022-04-25T07:34:50.822Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02\"][event_id@48577 hostname=\"jla-02\" uuid=\"1848d8a1-2f08-4a1e-bec4-ff9e6dd92553\" source=\"source\" unixtime=\"1650872090822\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872090\"] 470647 [Thread-3] INFO com.teragrep.jla_02.Logback Daily - Logback-daily says hi.".getBytes(StandardCharsets.UTF_8) + ) + ); + + consumer.addRecord(new ConsumerRecord<>(topicName, + partition, + 3L, + "2022-04-25T07:34:50.822Z".getBytes(StandardCharsets.UTF_8), + "<12>1 2022-04-25T07:34:50.822Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02\"][event_id@48577 hostname=\"jla-02\" uuid=\"5e1a0398-c2a0-468d-a562-c3bb31f0f853\" source=\"source\" unixtime=\"1650872090822\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872090\"] 470646 [Thread-3] INFO com.teragrep.jla_02.Logback Audit - Logback-audit says hi.".getBytes(StandardCharsets.UTF_8) + ) + ); + + consumer.addRecord(new ConsumerRecord<>(topicName, + partition, + 4L, + "2022-04-25T07:34:50.822Z".getBytes(StandardCharsets.UTF_8), + "<12>1 2022-04-25T07:34:50.822Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02\"][event_id@48577 hostname=\"jla-02\" uuid=\"6268c3a2-5bda-427f-acce-29416eb445f4\" source=\"source\" unixtime=\"1650872090822\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872090\"] 470647 [Thread-3] INFO com.teragrep.jla_02.Logback Metric - Logback-metric says hi.".getBytes(StandardCharsets.UTF_8) + ) + ); + + consumer.addRecord(new ConsumerRecord<>(topicName, + partition, + 5L, + "2022-04-25T07:34:52.238Z".getBytes(StandardCharsets.UTF_8), + "<12>1 2022-04-25T07:34:52.238Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"b500dcaf-1101-4000-b6b9-bfb052ddbf86\" source=\"source\" unixtime=\"1650872092238\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.238 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info audit says hi!]".getBytes(StandardCharsets.UTF_8) + ) + ); + + consumer.addRecord(new ConsumerRecord<>(topicName, + partition, + 6L, + "2022-04-25T07:34:52.239Z".getBytes(StandardCharsets.UTF_8), + "<12>1 2022-04-25T07:34:52.239Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"05363122-51ac-4c0b-a681-f5868081f56d\" source=\"source\" unixtime=\"1650872092239\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info daily says hi!]".getBytes(StandardCharsets.UTF_8) + ) + ); + + consumer.addRecord(new ConsumerRecord<>(topicName, + partition, + 7L, + "2022-04-25T07:34:52.239Z".getBytes(StandardCharsets.UTF_8), + "<12>1 2022-04-25T07:34:52.239Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"7bbcd843-b795-4c14-b4a1-95f5d445cbcd\" source=\"source\" unixtime=\"1650872092239\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info metric says hi!]".getBytes(StandardCharsets.UTF_8) + ) + ); + + consumer.addRecord(new ConsumerRecord<>(topicName, + partition, + 8L, + "2022-04-25T07:34:52.240Z".getBytes(StandardCharsets.UTF_8), + "<12>1 2022-04-25T07:34:52.240Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"2bc0a9f9-237d-4656-b40a-3038aace37f0\" source=\"source\" unixtime=\"1650872092240\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn audit says hi!]".getBytes(StandardCharsets.UTF_8) + ) + ); + + consumer.addRecord(new ConsumerRecord<>(topicName, + partition, + 9L, + "2022-04-25T07:34:52.240Z".getBytes(StandardCharsets.UTF_8), + "<12>1 2022-04-25T07:34:52.240Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"ecf61e8d-e3a7-48ef-9b73-3c5a5243d2e6\" source=\"source\" unixtime=\"1650872092240\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn daily says hi!]".getBytes(StandardCharsets.UTF_8) + ) + ); + + consumer.addRecord(new ConsumerRecord<>(topicName, + partition, + 10L, + "2022-04-25T07:34:52.241Z".getBytes(StandardCharsets.UTF_8), + "<12>1 2022-04-25T07:34:52.241Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"bf101d5a-e816-4f51-b132-97f8e3431f8e\" source=\"source\" unixtime=\"1650872092241\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.241 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn metric says hi!]".getBytes(StandardCharsets.UTF_8) + ) + ); + + consumer.addRecord(new ConsumerRecord<>(topicName, + partition, + 11L, + "2022-04-25T07:34:52.241Z".getBytes(StandardCharsets.UTF_8), + "<12>1 2022-04-25T07:34:52.241Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"ef94d9e9-3c44-4892-b5a6-bf361d13ff97\" source=\"source\" unixtime=\"1650872092241\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.241 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error audit says hi!]".getBytes(StandardCharsets.UTF_8) + ) + ); + + consumer.addRecord(new ConsumerRecord<>(topicName, + partition, + 12L, + "2022-04-25T07:34:52.242Z".getBytes(StandardCharsets.UTF_8), + "<12>1 2022-04-25T07:34:52.242Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"5bce6e3d-767d-44b4-a044-6c4872f8f2b5\" source=\"source\" unixtime=\"1650872092242\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.242 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error daily says hi!]".getBytes(StandardCharsets.UTF_8) + ) + ); + + consumer.addRecord(new ConsumerRecord<>(topicName, + partition, + 13L, + "2022-04-25T07:34:52.243Z".getBytes(StandardCharsets.UTF_8), + "<12>1 2022-04-25T07:34:52.243Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"3bb55ce4-0ea7-413a-b403-28b174d7ac99\" source=\"source\" unixtime=\"1650872092243\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.243 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error metric says hi!]".getBytes(StandardCharsets.UTF_8) + ) + ); + } } From faefd9bd06937906734da92fddff986295d9b816 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Mon, 22 Jan 2024 15:31:10 +0200 Subject: [PATCH 028/146] Altered Kafka MockConsumer to use subscription instead of assign method. --- .../kafka/MockKafkaConsumerFactoryTemp.java | 14 +---- .../teragrep/cfe_39/KafkaConsumerTest.java | 56 ++++++++++--------- 2 files changed, 32 insertions(+), 38 deletions(-) diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java index 6cf795b6..7c61405c 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java @@ -48,7 +48,6 @@ private static void generateEvents(MockConsumer consumer, String "<12>1 2022-04-25T07:34:50.822Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02\"][event_id@48577 hostname=\"jla-02\" uuid=\"1848d8a1-2f08-4a1e-bec4-ff9e6dd92553\" source=\"source\" unixtime=\"1650872090822\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872090\"] 470647 [Thread-3] INFO com.teragrep.jla_02.Logback Daily - Logback-daily says hi.".getBytes(StandardCharsets.UTF_8) ) ); - consumer.addRecord(new ConsumerRecord<>(topicName, partition, 3L, @@ -56,7 +55,6 @@ private static void generateEvents(MockConsumer consumer, String "<12>1 2022-04-25T07:34:50.822Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02\"][event_id@48577 hostname=\"jla-02\" uuid=\"5e1a0398-c2a0-468d-a562-c3bb31f0f853\" source=\"source\" unixtime=\"1650872090822\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872090\"] 470646 [Thread-3] INFO com.teragrep.jla_02.Logback Audit - Logback-audit says hi.".getBytes(StandardCharsets.UTF_8) ) ); - consumer.addRecord(new ConsumerRecord<>(topicName, partition, 4L, @@ -64,7 +62,6 @@ private static void generateEvents(MockConsumer consumer, String "<12>1 2022-04-25T07:34:50.822Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02\"][event_id@48577 hostname=\"jla-02\" uuid=\"6268c3a2-5bda-427f-acce-29416eb445f4\" source=\"source\" unixtime=\"1650872090822\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872090\"] 470647 [Thread-3] INFO com.teragrep.jla_02.Logback Metric - Logback-metric says hi.".getBytes(StandardCharsets.UTF_8) ) ); - consumer.addRecord(new ConsumerRecord<>(topicName, partition, 5L, @@ -72,7 +69,6 @@ private static void generateEvents(MockConsumer consumer, String "<12>1 2022-04-25T07:34:52.238Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"b500dcaf-1101-4000-b6b9-bfb052ddbf86\" source=\"source\" unixtime=\"1650872092238\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.238 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info audit says hi!]".getBytes(StandardCharsets.UTF_8) ) ); - consumer.addRecord(new ConsumerRecord<>(topicName, partition, 6L, @@ -88,7 +84,6 @@ private static void generateEvents(MockConsumer consumer, String "<12>1 2022-04-25T07:34:52.239Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"7bbcd843-b795-4c14-b4a1-95f5d445cbcd\" source=\"source\" unixtime=\"1650872092239\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info metric says hi!]".getBytes(StandardCharsets.UTF_8) ) ); - consumer.addRecord(new ConsumerRecord<>(topicName, partition, 8L, @@ -96,7 +91,6 @@ private static void generateEvents(MockConsumer consumer, String "<12>1 2022-04-25T07:34:52.240Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"2bc0a9f9-237d-4656-b40a-3038aace37f0\" source=\"source\" unixtime=\"1650872092240\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn audit says hi!]".getBytes(StandardCharsets.UTF_8) ) ); - consumer.addRecord(new ConsumerRecord<>(topicName, partition, 9L, @@ -104,7 +98,6 @@ private static void generateEvents(MockConsumer consumer, String "<12>1 2022-04-25T07:34:52.240Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"ecf61e8d-e3a7-48ef-9b73-3c5a5243d2e6\" source=\"source\" unixtime=\"1650872092240\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn daily says hi!]".getBytes(StandardCharsets.UTF_8) ) ); - consumer.addRecord(new ConsumerRecord<>(topicName, partition, 10L, @@ -112,7 +105,6 @@ private static void generateEvents(MockConsumer consumer, String "<12>1 2022-04-25T07:34:52.241Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"bf101d5a-e816-4f51-b132-97f8e3431f8e\" source=\"source\" unixtime=\"1650872092241\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.241 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn metric says hi!]".getBytes(StandardCharsets.UTF_8) ) ); - consumer.addRecord(new ConsumerRecord<>(topicName, partition, 11L, @@ -120,7 +112,6 @@ private static void generateEvents(MockConsumer consumer, String "<12>1 2022-04-25T07:34:52.241Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"ef94d9e9-3c44-4892-b5a6-bf361d13ff97\" source=\"source\" unixtime=\"1650872092241\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.241 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error audit says hi!]".getBytes(StandardCharsets.UTF_8) ) ); - consumer.addRecord(new ConsumerRecord<>(topicName, partition, 12L, @@ -128,7 +119,6 @@ private static void generateEvents(MockConsumer consumer, String "<12>1 2022-04-25T07:34:52.242Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"5bce6e3d-767d-44b4-a044-6c4872f8f2b5\" source=\"source\" unixtime=\"1650872092242\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.242 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error daily says hi!]".getBytes(StandardCharsets.UTF_8) ) ); - consumer.addRecord(new ConsumerRecord<>(topicName, partition, 13L, @@ -156,10 +146,12 @@ public static Consumer getConsumer() { endOffsets.put(topicPartition, 14L); mockPartitionInfo.add(new PartitionInfo("testConsumerTopic", i, null, null, null)); } - consumer.assign(topicPartitions); + consumer.subscribe(Collections.singletonList("testConsumerTopic")); // subscribe + // consumer.assign(topicPartitions); // assign consumer.updateBeginningOffsets(beginningOffsets); //insert stuff + consumer.rebalance(topicPartitions); // subscribe for (TopicPartition a : topicPartitions) { generateEvents(consumer, a.topic(), a.partition()); // The ordering in this loop is fine, goes from 0 to 9 in correct order. } diff --git a/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java b/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java index 332e8ac6..8d971e80 100644 --- a/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java +++ b/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java @@ -44,7 +44,7 @@ public void configTest() { } - // @Test + @Test public void kafkaAndAvroFullTest() throws InterruptedException { Config config = null; try { @@ -56,11 +56,11 @@ public void kafkaAndAvroFullTest() throws InterruptedException { System.out.println("Got invalid config: " + e); System.exit(1); } - config.setMaximumFileSize(8500); // 10 loops (140 records) are in use at the moment, and that is sized at 36,102 bits. + config.setMaximumFileSize(3000); // 10 loops (140 records) are in use at the moment, and that is sized at 36,102 bits. KafkaController kafkaController = new KafkaController(config); kafkaController.run(); try { - int counter = avroReader(1, 5); + int counter = avroReader(1, 2); Assertions.assertEquals(140, counter); } catch (IOException e) { throw new RuntimeException(e); @@ -75,24 +75,26 @@ public int avroReader(int start, int end) throws IOException { Path queueDirectory = Paths.get(config.getQueueDirectory()); int counter = 0; int looper = 0; - for (int i = start; i<=end; i++) { - File syslogFile = new File( - queueDirectory.toAbsolutePath() - + File.separator - + config.getQueueNamePrefix() - + "." - + i - ); - DatumReader userDatumReader = new SpecificDatumReader<>(SyslogRecord.class); - try (DataFileReader dataFileReader = new DataFileReader<>(syslogFile, userDatumReader)) { - SyslogRecord user = null; - int partitionCounter = 9; // The partitions are indexed from 0 to 9 when 10 loops are used in MockKafkaConsumerFactoryTemp. - while (dataFileReader.hasNext()) { - user = dataFileReader.next(user); - System.out.println(syslogFile.getPath()); - System.out.println(user); - counter++; - // All the mock data is generated from a set of 14 records. + for (int j = 0; j <= 9; j++) { + for (int i = start; i <= end; i++) { + File syslogFile = new File( + queueDirectory.toAbsolutePath() + + File.separator + + "testConsumerTopic" + + j + + "." + + i + ); + DatumReader userDatumReader = new SpecificDatumReader<>(SyslogRecord.class); + try (DataFileReader dataFileReader = new DataFileReader<>(syslogFile, userDatumReader)) { + SyslogRecord user = null; + int partitionCounter = 9; // The partitions are indexed from 0 to 9 when 10 loops are used in MockKafkaConsumerFactoryTemp. + while (dataFileReader.hasNext()) { + user = dataFileReader.next(user); + System.out.println(syslogFile.getPath()); + System.out.println(user); + counter++; + // All the mock data is generated from a set of 14 records. /*if (looper <= 0) { // FIXME: The partition ordering is wrong in kafkaconsumer. Must be fixed so the avro serialization works properly with correct filenames for HDFS. Assertions.assertEquals("{\"timestamp\": 1650872090804000, \"message\": \"[WARN] 2022-04-25 07:34:50,804 com.teragrep.jla_02.Log4j Log - Log4j warn says hi!\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \""+partitionCounter+"\", \"offset\": 0, \"origin\": \"jla-02.default\"}", user.toString()); @@ -138,6 +140,7 @@ public int avroReader(int start, int end) throws IOException { looper = 0; partitionCounter--; }*/ + } } } } @@ -148,15 +151,14 @@ public int avroReader(int start, int end) throws IOException { // @Test public void debugger() { - - /*try { - int counter = avroReader(1, 5); + try { + int counter = avroReader(1, 2); Assertions.assertEquals(140, counter); } catch (IOException e) { throw new RuntimeException(e); - }*/ + } - // TODO: DEBUG THE PARTITION ORDER! + /*// TODO: DEBUG THE PARTITION ORDER! // rewrite the key values for mock data generation. int amountofloops = 10; // number of loops for adding partitions/records to the mock consumer topic. Each loop adds a new partition of 14 records. 17777 loops results in file size slightly above 64M. 10 loops is sized at 36,102 bits. @@ -197,7 +199,7 @@ public void debugger() { for (TopicPartition a : checkAssignmentOder) { Assertions.assertEquals(new TopicPartition("testConsumerTopic", looper), a); looper++; - } + }*/ From 484b6aa4624011135587a268c9c85c7c7a9a21a3 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Tue, 23 Jan 2024 14:09:44 +0200 Subject: [PATCH 029/146] Implementing fix to consumer group creation for partitions. --- README.adoc | 4 +- .../consumers/kafka/KafkaController.java | 52 ++++++++++++++++--- .../kafka/MockKafkaConsumerFactoryTemp.java | 6 ++- .../consumers/kafka/SyslogAvroWriter.java | 1 + 4 files changed, 53 insertions(+), 10 deletions(-) diff --git a/README.adoc b/README.adoc index df8dbc33..05db87ac 100644 --- a/README.adoc +++ b/README.adoc @@ -13,9 +13,11 @@ See the official documentation on https://docs.teragrep.com[docs.teragrep.com]. ## How to [compile/use/implement] -// TODO: add instructions how people can start to use your project +// TODO: add instructions how people can start to use your project, also add more information on the required configuration files. mvn clean install +config.jaas, application.properties and log4j2.properties files have to created to use this module. The files must be placed in the working directory. + ## Contributing diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java index 545d9224..eb6ab295 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java @@ -102,7 +102,7 @@ public void run() throws InterruptedException { } // Creates kafka topic consumer based on input parameters. - private void createReader(String topic, List topicCounters) throws SQLException { + private void createReader(String topic, List listPartitionInfo, List topicCounters) throws SQLException { // Create a new topicCounter object for the topic that has not been added to topicCounters-list yet. TopicCounter topicCounter = new TopicCounter(topic); @@ -126,24 +126,44 @@ private void createReader(String topic, List topicCounters) throws config.getKafkaConsumerProperties(), output ); + long totalRecords = topicCounter.getTotalRecords(); + +// Thread readThread = new Thread(null, readCoordinator, topic); // Starts the thread with readCoordinator that creates the consumer and subscribes to the topic. +// threads.add(readThread); +// readThread.start(); // Starts the thread, in other words proceeds to call run() function of ReadCoordinator. + // Every consumer is run in a separate thread. - Thread readThread = new Thread(null, readCoordinator, topic); - threads.add(readThread); - readThread.start(); + // FIXME: Exception in thread "testConsumerTopic" java.nio.channels.OverlappingFileLockException. The cause is that the consumers are accessing the same partition for some reason when the partitions are supposed to be assigned to a single consumer. + // In other words the consumers are trying to store records to the same AVRO-file. The problem us most likely in the mock consumer side. That thing is confusing when trying to implement consumer groups. + int numOfThreads = Math.min(1, listPartitionInfo.size()); // Makes sure that more consumers are not assigned to the topic than there are partitions available on the topic. + for (int testi = 1; numOfThreads >= testi; testi++) { + Thread readThread = new Thread(null, readCoordinator, topic); // Starts the thread with readCoordinator that creates the consumer and subscribes to the topic. + threads.add(readThread); + readThread.start(); // Starts the thread, in other words proceeds to call run() function of ReadCoordinator. + } + } private void topicScan(DurationStatistics durationStatistics, List topicCounters) { Map> listTopics = kafkaConsumer.listTopics(Duration.ofSeconds(60)); // Topics can be fetched from mock consumer if the consumer has been updated separately with the partition info. Pattern topicsRegex = Pattern.compile(config.getQueueTopicPattern()); // Mock consumer has the partitions in this format: queueTopicPattern=^testConsumerTopic-*$ - // Find the topics available in Kafka based on given QueueTopicPattern, both active and in-active. // Check how partitions are handled, need to allow using consumer groups for partition read assignments. aka. load balancing Set foundTopics = new HashSet<>(); + + // TODO: 1. Add functionality so the partition information is also fetched for the queried topics. At the moment only the topic names are fetched. + Map> foundPartitions = new HashMap<>(); + + for (Map.Entry> entry : listTopics.entrySet()) { Matcher matcher = topicsRegex.matcher(entry.getKey()); if (matcher.matches()) { foundTopics.add(entry.getKey()); + + // TODO: 2. Add functionality so the partition information is also fetched for the queried topics. At the moment only the topic names are fetched. + foundPartitions.put(entry.getKey(), entry.getValue()); + } } @@ -153,9 +173,27 @@ private void topicScan(DurationStatistics durationStatistics, List // subtract currently active topics from found topics foundTopics.removeAll(activeTopics); + // TODO: 3. Subtract currently active partitions from found partitions + for (String topic_name : activeTopics) { + foundPartitions.remove(topic_name); // removes the partitions from the list based on the topic name. + } + + + // TODO: Activate all the found in-active topics, in other words create individual consumers for all of them using the createReader()-function. + foundPartitions.forEach((k, v) -> { + LOGGER.info("Activating topic <"+k+">"); + try { + createReader(k, v, topicCounters); + activeTopics.add(k); + durationStatistics.addAndGetThreads(1); + } catch (SQLException sqlException) { + LOGGER.error("Topic <"+k+"> not activated due to reader creation error: " + sqlException); + } + }); + durationStatistics.report(); // Activate all the found in-active topics, in other words create individual consumers for all of them using the createReader()-function. - for (String topic : foundTopics) { +/* for (String topic : foundTopics) { LOGGER.info("Activating topic <"+topic+">"); try { createReader(topic, topicCounters); @@ -166,7 +204,7 @@ private void topicScan(DurationStatistics durationStatistics, List LOGGER.error("Topic <"+topic+"> not activated due to reader creation error: " + sqlException); } } - durationStatistics.report(); + durationStatistics.report();*/ } } diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java index 7c61405c..07f4238e 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java @@ -151,13 +151,15 @@ public static Consumer getConsumer() { consumer.updateBeginningOffsets(beginningOffsets); //insert stuff - consumer.rebalance(topicPartitions); // subscribe + consumer.rebalance(topicPartitions); // needed for subscribe for (TopicPartition a : topicPartitions) { - generateEvents(consumer, a.topic(), a.partition()); // The ordering in this loop is fine, goes from 0 to 9 in correct order. + generateEvents(consumer, a.topic(), a.partition()); } consumer.updateEndOffsets(endOffsets); consumer.updatePartitions("testConsumerTopic", mockPartitionInfo); return consumer; + // TODO: Check how to implement consumer group in mockconsumer. The mockconsumer has consumer.groupMetadata() method available so it should be possible even though its not thread safe. + // The code for starting consumers in separate threads is located in KafkaController.java line 132. } } \ No newline at end of file diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java index 7eee088f..233d11da 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java @@ -27,6 +27,7 @@ class SyslogAvroWriter implements AutoCloseable { syncableFileOutputStream = new SyncableFileOutputStream(syslogFile); + System.out.println("debugging syslogFile, path is: " + syslogFile.getPath()); syncableFileOutputStream.getChannel().tryLock(); if (syslogFile.length() == 0) { From 09042800da7d1b7d050cab738e72c11bb44c3e3b Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Wed, 24 Jan 2024 10:27:12 +0200 Subject: [PATCH 030/146] Added config parameter for defining the number of consumers in a consumer group. --- src/main/java/com/teragrep/cfe_39/Config.java | 5 +++++ .../com/teragrep/cfe_39/consumers/kafka/KafkaController.java | 4 +++- 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/src/main/java/com/teragrep/cfe_39/Config.java b/src/main/java/com/teragrep/cfe_39/Config.java index f3cd4ba4..c7fc4aa0 100644 --- a/src/main/java/com/teragrep/cfe_39/Config.java +++ b/src/main/java/com/teragrep/cfe_39/Config.java @@ -46,6 +46,7 @@ public class Config { private final String kerberosKeytabPath; private final String kerberosTestMode; private long maximumFileSize; + private final int numOfConsumers; // TODO: Set up configuration check for important parameters. @@ -113,6 +114,7 @@ public class Config { // kafka this.queueTopicPattern = properties.getProperty("queueTopicPattern", "^.*$"); + this.numOfConsumers = Integer.parseInt(properties.getProperty("numOfConsumers", "1")); this.kafkaConsumerProperties = loadSubProperties(properties, "consumer."); String loginConfig = properties.getProperty("java.security.auth.login.config"); @@ -237,4 +239,7 @@ public long getMaximumFileSize() { public void setMaximumFileSize(long maximumFileSize) { this.maximumFileSize = maximumFileSize; } + public int getNumOfConsumers() { + return numOfConsumers; + } } \ No newline at end of file diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java index eb6ab295..2edebddc 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java @@ -56,12 +56,14 @@ public class KafkaController { private final DurationStatistics durationStatistics = new DurationStatistics(); private boolean keepRunning; private boolean useMockKafkaConsumer; + private final int numOfConsumers; // private final MetricRegistry metricRegistry = new MetricRegistry(); public KafkaController(Config config) { keepRunning = true; this.config = config; Properties readerKafkaProperties = config.getKafkaConsumerProperties(); + this.numOfConsumers = config.getNumOfConsumers(); // TODO: Add config parametrization. this.useMockKafkaConsumer = Boolean.parseBoolean( readerKafkaProperties.getProperty("useMockKafkaConsumer", "false") ); @@ -136,7 +138,7 @@ private void createReader(String topic, List listPartitionInfo, L // Every consumer is run in a separate thread. // FIXME: Exception in thread "testConsumerTopic" java.nio.channels.OverlappingFileLockException. The cause is that the consumers are accessing the same partition for some reason when the partitions are supposed to be assigned to a single consumer. // In other words the consumers are trying to store records to the same AVRO-file. The problem us most likely in the mock consumer side. That thing is confusing when trying to implement consumer groups. - int numOfThreads = Math.min(1, listPartitionInfo.size()); // Makes sure that more consumers are not assigned to the topic than there are partitions available on the topic. + int numOfThreads = Math.min(numOfConsumers, listPartitionInfo.size()); // Makes sure that more consumers are not assigned to the topic than there are partitions available on the topic. for (int testi = 1; numOfThreads >= testi; testi++) { Thread readThread = new Thread(null, readCoordinator, topic); // Starts the thread with readCoordinator that creates the consumer and subscribes to the topic. threads.add(readThread); From a08b61066e70ccef3f8e9ffc177f0a0b084ceca9 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Thu, 25 Jan 2024 11:26:31 +0200 Subject: [PATCH 031/146] Fixed consumer group generation and its tests. Added example configuration files to test folder. --- src/main/java/com/teragrep/cfe_39/Config.java | 2 +- .../consumers/kafka/DatabaseOutput.java | 8 +-- .../consumers/kafka/KafkaController.java | 61 ++++++++----------- .../kafka/MockKafkaConsumerFactoryTemp.java | 49 ++++++++++++--- .../consumers/kafka/ReadCoordinatorTemp.java | 13 ++-- .../teragrep/cfe_39/application.properties | 42 +++++++++++++ src/test/java/com/teragrep/cfe_39/config.jaas | 9 +++ .../com/teragrep/cfe_39/log4j2.properties | 10 +++ 8 files changed, 141 insertions(+), 53 deletions(-) create mode 100644 src/test/java/com/teragrep/cfe_39/application.properties create mode 100644 src/test/java/com/teragrep/cfe_39/config.jaas create mode 100644 src/test/java/com/teragrep/cfe_39/log4j2.properties diff --git a/src/main/java/com/teragrep/cfe_39/Config.java b/src/main/java/com/teragrep/cfe_39/Config.java index c7fc4aa0..f4c9df4a 100644 --- a/src/main/java/com/teragrep/cfe_39/Config.java +++ b/src/main/java/com/teragrep/cfe_39/Config.java @@ -48,7 +48,7 @@ public class Config { private long maximumFileSize; private final int numOfConsumers; - // TODO: Set up configuration check for important parameters. + // TODO: Set up configuration check for important parameters. Remove old unused parameters. Config() throws IOException { Properties properties = new Properties(); diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java index ac15db4f..f7dbe083 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java @@ -130,9 +130,9 @@ private long rfc3339ToEpoch(ZonedDateTime zonedDateTime) { } // Input parameter is a list of RecordOffsetObjects. Each object contains a record and its metadata (topic, partition and offset). - // TODO: Alter the processing so each partition will get their exclusive files. - // The target where the record is stored in HDFS is based on the topic, partition and offset. ie. topic_name/0.123456 where offset is 123456. - // AVRO-file with a path/name that starts with topic_name/0.X should only contain records from the 0th partition of topic named topic_name, topic_name/1.X should only contain records from 1st partition, etc. + // Each partition will get their exclusive files. + // The target where the record is stored in HDFS is based on the topic, partition and offset. ie. topic_name/0.123456 where offset is 123456. + // AVRO-file with a path/name that starts with topic_name/0.X should only contain records from the 0th partition of topic named topic_name, topic_name/1.X should only contain records from 1st partition, etc. @Override public void accept(List recordOffsetObjectList) { long thisTime = Instant.now().toEpochMilli(); @@ -150,7 +150,7 @@ public void accept(List recordOffsetObjectList) { // every recordOffsetObject.record on the recordOffsetObjectList basically represents a rlo_09 WriteCoordinator.accept(byte[] bytes) when the list is gone through in a loop. RecordOffsetObject lastObject = null; long start = Instant.now().toEpochMilli(); // Starts measuring performance here. Measures how long it takes to process the whole recordOffsetObjectList. - // TODO: This loop goes through all the records of the mock data in a single session. + // This loop goes through all the records of the mock data in a single session. for (RecordOffsetObject recordOffsetObject : recordOffsetObjectList) { // Initializing syslogAvroWriter. if (syslogAvroWriter == null) { diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java index 2edebddc..43c7ba22 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java @@ -19,6 +19,8 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; +// TODO: Clean up comments. + public class KafkaController { // rlo_09 KafkaReader is the code that I should take a look at. ReadCoordinator alone won't allow access to the kafka offsets, it must be done in KafkaReader. // ReadCoordinator uses the KafkaReader, but it's set as private and there are no functions for accessing it through ReadCoordinator. @@ -63,12 +65,12 @@ public KafkaController(Config config) { keepRunning = true; this.config = config; Properties readerKafkaProperties = config.getKafkaConsumerProperties(); - this.numOfConsumers = config.getNumOfConsumers(); // TODO: Add config parametrization. + this.numOfConsumers = config.getNumOfConsumers(); this.useMockKafkaConsumer = Boolean.parseBoolean( readerKafkaProperties.getProperty("useMockKafkaConsumer", "false") ); if (useMockKafkaConsumer) { - this.kafkaConsumer = MockKafkaConsumerFactoryTemp.getConsumer(); + this.kafkaConsumer = MockKafkaConsumerFactoryTemp.getConsumer(0); // Used only for scanning the available topics. } else { this.kafkaConsumer = new KafkaConsumer<>(config.getKafkaConsumerProperties(), new ByteArrayDeserializer(), new ByteArrayDeserializer()); } @@ -111,36 +113,27 @@ private void createReader(String topic, List listPartitionInfo, L // Add the new topicCounter object to the list. topicCounters.add(topicCounter); - // DatabaseOutput handles transferring the consumed data to storage (S3, mariadb, HDFS, etc.) - // Kafka offset tracking must be included here. - // Topic is figured out in topicScan so the offsets for the topic should be figured out here. - Consumer> output = new DatabaseOutput( - config, // Configuration settings - topic, // String, the name of the topic - durationStatistics, // RuntimeStatistics object from metrics - topicCounter // TopicCounter object from metrics - ); - - // The kafka offsets must be passed to HDFS. The consumer must also be set to manual commits so the HDFS can handle managing the commit offsets within the HDFS filenames. - // plain rlo_09.ReadCoordinator won't give access to offset values. Implementing custom rlo_09 code in the package to achieve offset access. - ReadCoordinatorTemp readCoordinator = new ReadCoordinatorTemp( - topic, - config.getKafkaConsumerProperties(), - output - ); - long totalRecords = topicCounter.getTotalRecords(); - -// Thread readThread = new Thread(null, readCoordinator, topic); // Starts the thread with readCoordinator that creates the consumer and subscribes to the topic. -// threads.add(readThread); -// readThread.start(); // Starts the thread, in other words proceeds to call run() function of ReadCoordinator. - - // Every consumer is run in a separate thread. - // FIXME: Exception in thread "testConsumerTopic" java.nio.channels.OverlappingFileLockException. The cause is that the consumers are accessing the same partition for some reason when the partitions are supposed to be assigned to a single consumer. - // In other words the consumers are trying to store records to the same AVRO-file. The problem us most likely in the mock consumer side. That thing is confusing when trying to implement consumer groups. - int numOfThreads = Math.min(numOfConsumers, listPartitionInfo.size()); // Makes sure that more consumers are not assigned to the topic than there are partitions available on the topic. + // Consumer group is also handled here, and each consumer of the group runs on separate thread. + int numOfThreads = Math.min(numOfConsumers, listPartitionInfo.size()); // Makes sure that there aren't more consumers than available partitions in the consumer group. for (int testi = 1; numOfThreads >= testi; testi++) { - Thread readThread = new Thread(null, readCoordinator, topic); // Starts the thread with readCoordinator that creates the consumer and subscribes to the topic. + // DatabaseOutput handles transferring the consumed data to storage (S3, mariadb, HDFS, etc.) + // Kafka offset tracking must be included here. + // Topic is figured out in topicScan so the offsets for the topic should be figured out here. + Consumer> output = new DatabaseOutput( + config, // Configuration settings + topic, // String, the name of the topic + durationStatistics, // RuntimeStatistics object from metrics + topicCounter // TopicCounter object from metrics + ); + // The kafka offsets must be passed to HDFS. The consumer must also be set to manual commits so the HDFS can handle managing the commit offsets within the HDFS filenames. + // plain rlo_09.ReadCoordinator won't give access to offset values. Implementing custom rlo_09 code in the package to achieve offset access. + ReadCoordinatorTemp readCoordinator = new ReadCoordinatorTemp( + topic, + config.getKafkaConsumerProperties(), + output + ); + Thread readThread = new Thread(null, readCoordinator, topic+testi); // Starts the thread with readCoordinator that creates the consumer and subscribes to the topic. threads.add(readThread); readThread.start(); // Starts the thread, in other words proceeds to call run() function of ReadCoordinator. } @@ -154,7 +147,7 @@ private void topicScan(DurationStatistics durationStatistics, List // Check how partitions are handled, need to allow using consumer groups for partition read assignments. aka. load balancing Set foundTopics = new HashSet<>(); - // TODO: 1. Add functionality so the partition information is also fetched for the queried topics. At the moment only the topic names are fetched. + // 1. Add functionality so the partition information is also fetched for the queried topics. At the moment only the topic names are fetched. Map> foundPartitions = new HashMap<>(); @@ -163,7 +156,7 @@ private void topicScan(DurationStatistics durationStatistics, List if (matcher.matches()) { foundTopics.add(entry.getKey()); - // TODO: 2. Add functionality so the partition information is also fetched for the queried topics. At the moment only the topic names are fetched. + // 2. Add functionality so the partition information is also fetched for the queried topics. At the moment only the topic names are fetched. foundPartitions.put(entry.getKey(), entry.getValue()); } @@ -175,13 +168,13 @@ private void topicScan(DurationStatistics durationStatistics, List // subtract currently active topics from found topics foundTopics.removeAll(activeTopics); - // TODO: 3. Subtract currently active partitions from found partitions + // 3. Subtract currently active partitions from found partitions for (String topic_name : activeTopics) { foundPartitions.remove(topic_name); // removes the partitions from the list based on the topic name. } - // TODO: Activate all the found in-active topics, in other words create individual consumers for all of them using the createReader()-function. + // Activate all the found in-active topics, in other words create individual consumers for all of them using the createReader()-function. foundPartitions.forEach((k, v) -> { LOGGER.info("Activating topic <"+k+">"); try { diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java index 07f4238e..f251ead3 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java @@ -128,7 +128,8 @@ private static void generateEvents(MockConsumer consumer, String ); } - public static Consumer getConsumer() { + // Can initialize topic scan with all partitions available when the input parameter is 0. Consumer is manually assigned to specific partitions depending on the threadnum parameter. For example on threadnum 1 consumer has odd numbered partitions assigned to it and threadnum 2 has the even numbered partitions. + public static Consumer getConsumer(int threadnum) { LOGGER.warn("useMockKafkaConsumer is set, using MockKafkaConsumer"); int amountofloops = 10; // number of loops for adding partitions/records to the mock consumer topic. Each loop adds a new partition of 14 records. 17777 loops results in file size slightly above 64M. 10 loops is sized at 36,102 bits. @@ -146,20 +147,52 @@ public static Consumer getConsumer() { endOffsets.put(topicPartition, 14L); mockPartitionInfo.add(new PartitionInfo("testConsumerTopic", i, null, null, null)); } - consumer.subscribe(Collections.singletonList("testConsumerTopic")); // subscribe - // consumer.assign(topicPartitions); // assign + // consumer.subscribe(Collections.singletonList("testConsumerTopic")); // subscribe + + if (threadnum == 1) { + List oddTopicPartitions = new ArrayList<>(); + for (TopicPartition a : topicPartitions) { + if(((a.partition() + 1) % 2) == 0) { + oddTopicPartitions.add(a); + } + } + consumer.assign(oddTopicPartitions); // assign + for (TopicPartition a : topicPartitions) { + if(((a.partition() + 1) % 2) == 0) { + generateEvents(consumer, a.topic(), a.partition()); + } + } + } else if (threadnum == 2) { + List evenTopicPartitions = new ArrayList<>(); + for (TopicPartition a : topicPartitions) { + if(((a.partition() + 1) % 2) != 0) { + evenTopicPartitions.add(a); + } + } + consumer.assign(evenTopicPartitions); // assign + for (TopicPartition a : topicPartitions) { + if(((a.partition() + 1) % 2) != 0) { + generateEvents(consumer, a.topic(), a.partition()); + } + } + }else { + consumer.assign(topicPartitions); // assign + for (TopicPartition a : topicPartitions) { + generateEvents(consumer, a.topic(), a.partition()); + } + } + consumer.updateBeginningOffsets(beginningOffsets); //insert stuff - consumer.rebalance(topicPartitions); // needed for subscribe - for (TopicPartition a : topicPartitions) { + // consumer.rebalance(topicPartitions); // needed for subscribe + /*for (TopicPartition a : topicPartitions) { generateEvents(consumer, a.topic(), a.partition()); - } + }*/ consumer.updateEndOffsets(endOffsets); consumer.updatePartitions("testConsumerTopic", mockPartitionInfo); return consumer; - // TODO: Check how to implement consumer group in mockconsumer. The mockconsumer has consumer.groupMetadata() method available so it should be possible even though its not thread safe. - // The code for starting consumers in separate threads is located in KafkaController.java line 132. + // The code for starting consumers in separate threads is located in KafkaController.java line 138. } } \ No newline at end of file diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinatorTemp.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinatorTemp.java index b5e4c0d0..f51fdf95 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinatorTemp.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinatorTemp.java @@ -36,7 +36,12 @@ private KafkaReaderTemp createKafkaReader(Properties readerKafkaProperties, org.apache.kafka.clients.consumer.Consumer kafkaConsumer; if (useMockKafkaConsumer) { - kafkaConsumer = MockKafkaConsumerFactoryTemp.getConsumer(); + String name = Thread.currentThread().getName(); + if (Objects.equals(name, "testConsumerTopic1")) { + kafkaConsumer = MockKafkaConsumerFactoryTemp.getConsumer(1); + }else { + kafkaConsumer = MockKafkaConsumerFactoryTemp.getConsumer(2); + } } else { kafkaConsumer = new KafkaConsumer<>(readerKafkaProperties, new ByteArrayDeserializer(), new ByteArrayDeserializer()); kafkaConsumer.subscribe(Collections.singletonList(topic)); @@ -47,11 +52,7 @@ private KafkaReaderTemp createKafkaReader(Properties readerKafkaProperties, @Override public void run() { - boolean useMockKafkaConsumer = Boolean.parseBoolean( - readerKafkaProperties.getProperty("useMockKafkaConsumer", "false") - ); - - + boolean useMockKafkaConsumer = Boolean.parseBoolean(readerKafkaProperties.getProperty("useMockKafkaConsumer", "false")); try ( KafkaReaderTemp kafkaReader = createKafkaReader( readerKafkaProperties, diff --git a/src/test/java/com/teragrep/cfe_39/application.properties b/src/test/java/com/teragrep/cfe_39/application.properties new file mode 100644 index 00000000..80fe3bb8 --- /dev/null +++ b/src/test/java/com/teragrep/cfe_39/application.properties @@ -0,0 +1,42 @@ +# Kafka security configuration file +java.security.auth.login.config=/home/p000048u/IdeaProjects/cfe_39/etc/config.jaas +# Logger settings +log4j2.configurationFile=/home/p000048u/IdeaProjects/cfe_39/etc/log4j2.properties +# Galera replication and fragment sizes/unit +db.replicationEnabled=true +db.streamUnits=bytes +db.streamSize=64000000 +# Override table and partition default location? +db.overrideTableLocation=false +db.tableLocation=/path/to/storage +# Partitions to drop that are older than this, must be >0 +db.dropPartitionsOlderThanHours=6 +# Partitions to create in advance, must be >0 +db.createPartitionsInAdvanceHours=168 +# What topics are searched from kafka, regex +queueTopicPattern=^testConsumerTopic-*$ +# Number of consumers created to the consumer group +numOfConsumers=1 +# Kafka bootstrap servers - 127.0.0.1:9094,127.0.0.2:9094,127.0.0.3:9094 +consumer.bootstrap.servers=test +# Offset, should not be touched +consumer.auto.offset.reset=earliest +# Autocommit, should not be touched +consumer.enable.auto.commit=false +# Consumer group id, this is to track the progress of reading hte topic +consumer.group.id=cfe_39 +# Used security protocol and mechanism +consumer.security.protocol=SASL_PLAINTEXT +consumer.sasl.mechanism=PLAIN +# Maximum records per batch, note that too big number will cause massive load and can cause timeouts to trigger +consumer.max.poll.records=500 +# How much data can be fetched in one go +consumer.fetch.max.bytes=1073741820 +# How long for request before timing out. Note that too big max poll records size can cause this to trigger +consumer.request.timeout.ms=300000 +consumer.max.poll.interval.ms=300000 +# For testing only, remove for prod. +consumer.useMockKafkaConsumer=true +# AVRO +queueDirectory=/home/p000048u/IdeaProjects/cfe_39/etc/AVRO/ +queueNamePrefix=testingAVRO \ No newline at end of file diff --git a/src/test/java/com/teragrep/cfe_39/config.jaas b/src/test/java/com/teragrep/cfe_39/config.jaas new file mode 100644 index 00000000..045b8540 --- /dev/null +++ b/src/test/java/com/teragrep/cfe_39/config.jaas @@ -0,0 +1,9 @@ +KafkaServer { + org.apache.kafka.common.security.plain.PlainLoginModule required + username="admin" + password="admin" + user_admin="admin" + user_alice="alice" + user_bob="bob" + user_charlie="charlie"; +}; \ No newline at end of file diff --git a/src/test/java/com/teragrep/cfe_39/log4j2.properties b/src/test/java/com/teragrep/cfe_39/log4j2.properties new file mode 100644 index 00000000..9ec3d8ec --- /dev/null +++ b/src/test/java/com/teragrep/cfe_39/log4j2.properties @@ -0,0 +1,10 @@ +appender.console.type = Console +appender.console.name = ConsoleLogger +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = %d{HH:mm:ss.SSS} [%t] %-5level %logger{36} - %msg%n +logging.level.org.apache.kafka=WARN +logging.level.io.confluent.kafka=WARN +rootLogger.level = INFO +rootLogger.appenderRef.stdout.ref = ConsoleLogger +logger.kafka.name = org.apache.kafka +logger.kafka.level = warn \ No newline at end of file From 82bd6f5683b060a0d3453782c748495dccafa4fc Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Thu, 25 Jan 2024 15:59:31 +0200 Subject: [PATCH 032/146] Cleaned up and improved tests. --- .../consumers/kafka/KafkaController.java | 2 +- .../consumers/kafka/SyslogAvroWriter.java | 2 +- .../teragrep/cfe_39/KafkaConsumerTest.java | 245 ++---------------- 3 files changed, 20 insertions(+), 229 deletions(-) diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java index 43c7ba22..c11d894f 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java @@ -174,7 +174,7 @@ private void topicScan(DurationStatistics durationStatistics, List } - // Activate all the found in-active topics, in other words create individual consumers for all of them using the createReader()-function. + // Activate all the found in-active topics, in other words create consumer groups for all of them using the createReader()-function. foundPartitions.forEach((k, v) -> { LOGGER.info("Activating topic <"+k+">"); try { diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java index 233d11da..d08defb2 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java @@ -27,7 +27,7 @@ class SyslogAvroWriter implements AutoCloseable { syncableFileOutputStream = new SyncableFileOutputStream(syslogFile); - System.out.println("debugging syslogFile, path is: " + syslogFile.getPath()); + // System.out.println("debugging syslogFile, path is: " + syslogFile.getPath()); syncableFileOutputStream.getChannel().tryLock(); if (syslogFile.length() == 0) { diff --git a/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java b/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java index 8d971e80..fbb1672c 100644 --- a/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java +++ b/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java @@ -75,6 +75,7 @@ public int avroReader(int start, int end) throws IOException { Path queueDirectory = Paths.get(config.getQueueDirectory()); int counter = 0; int looper = 0; + int partitionCounter = 0; for (int j = 0; j <= 9; j++) { for (int i = start; i <= end; i++) { File syslogFile = new File( @@ -88,15 +89,13 @@ public int avroReader(int start, int end) throws IOException { DatumReader userDatumReader = new SpecificDatumReader<>(SyslogRecord.class); try (DataFileReader dataFileReader = new DataFileReader<>(syslogFile, userDatumReader)) { SyslogRecord user = null; - int partitionCounter = 9; // The partitions are indexed from 0 to 9 when 10 loops are used in MockKafkaConsumerFactoryTemp. while (dataFileReader.hasNext()) { user = dataFileReader.next(user); System.out.println(syslogFile.getPath()); System.out.println(user); counter++; // All the mock data is generated from a set of 14 records. - /*if (looper <= 0) { - // FIXME: The partition ordering is wrong in kafkaconsumer. Must be fixed so the avro serialization works properly with correct filenames for HDFS. + if (looper <= 0) { Assertions.assertEquals("{\"timestamp\": 1650872090804000, \"message\": \"[WARN] 2022-04-25 07:34:50,804 com.teragrep.jla_02.Log4j Log - Log4j warn says hi!\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \""+partitionCounter+"\", \"offset\": 0, \"origin\": \"jla-02.default\"}", user.toString()); looper++; } else if (looper == 1) { @@ -138,8 +137,8 @@ public int avroReader(int start, int end) throws IOException { } else { Assertions.assertEquals("{\"timestamp\": 1650872092243000, \"message\": \"25.04.2022 07:34:52.243 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error metric says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \""+partitionCounter+"\", \"offset\": 13, \"origin\": \"jla-02.default\"}", user.toString()); looper = 0; - partitionCounter--; - }*/ + partitionCounter++; + } } } } @@ -150,241 +149,33 @@ public int avroReader(int start, int end) throws IOException { // @Test public void debugger() { - try { int counter = avroReader(1, 2); Assertions.assertEquals(140, counter); } catch (IOException e) { throw new RuntimeException(e); } - - /*// TODO: DEBUG THE PARTITION ORDER! - // rewrite the key values for mock data generation. - - int amountofloops = 10; // number of loops for adding partitions/records to the mock consumer topic. Each loop adds a new partition of 14 records. 17777 loops results in file size slightly above 64M. 10 loops is sized at 36,102 bits. - final MockConsumer consumer; - consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); - List topicPartitions = new ArrayList<>(); - LinkedHashMap beginningOffsets = new LinkedHashMap<>(); - LinkedHashMap endOffsets = new LinkedHashMap<>(); - List mockPartitionInfo = new ArrayList<>(); - // generate the topic partitions and metadata first - for (int i = 0; i < amountofloops; i++) { - TopicPartition topicPartition = new TopicPartition("testConsumerTopic", i); - topicPartitions.add(topicPartition); - beginningOffsets.put(topicPartition, 0L); - endOffsets.put(topicPartition, 14L); - mockPartitionInfo.add(new PartitionInfo("testConsumerTopic", i, null, null, null)); - } - Collection testing = new ArrayList<>(topicPartitions); - consumer.subscribe(Collections.singletonList("testConsumerTopic")); - // consumer.assign(testing); // FIXME: PARTITIONS ARE IN WRONG ORDER IN kafkaConsumer, 7856341209. Should be 0123456789. - - consumer.updateBeginningOffsets(beginningOffsets); - - //insert stuff - for (TopicPartition a : topicPartitions) { - consumer.rebalance(Collections.singletonList(new TopicPartition("testConsumerTopic", 0))); - generateEvents(consumer, a.topic(), a.partition()); // The ordering in this loop is fine, goes from 0 to 9 in correct order. - } - - consumer.updateEndOffsets(endOffsets); - consumer.updatePartitions("testConsumerTopic", mockPartitionInfo); - - - // ASSERTIONS - Set checkAssignmentOder = consumer.assignment(); // for testing only - consumer.updateBeginningOffsets(beginningOffsets); - int looper = 0; - for (TopicPartition a : checkAssignmentOder) { - Assertions.assertEquals(new TopicPartition("testConsumerTopic", looper), a); - looper++; - }*/ - - - } // Deletes the avro-files that were created during testing. public void cleanup(Config config, int start, int end) { Path queueDirectory = Paths.get(config.getQueueDirectory()); - String queueNamePrefix = config.getQueueNamePrefix(); - for (int nextSequenceNumber = start; nextSequenceNumber <= end; nextSequenceNumber++) { - File file = new File( - queueDirectory.toAbsolutePath() - + File.separator - + queueNamePrefix - + "." - + nextSequenceNumber - ); - try { - boolean result = Files.deleteIfExists(file.toPath()); //surround it in try catch block - } catch (IOException e) { - throw new RuntimeException(e); - } - } - } - - - // @Test - public void kafkaConsumerTest() throws InterruptedException { - // The mock kafka consumer is enabled in the config. Now it should be possible to implement tests using it. https://www.baeldung.com/kafka-mockconsumer - // This code can be implemented as Main.main() function later. - Config config = null; - try { - config = new Config(); - } catch (IOException e){ - System.out.println("Can't load config: " + e); - System.exit(1); - } catch (IllegalArgumentException e) { - System.out.println("Got invalid config: " + e); - System.exit(1); - } - config.setMaximumFileSize(3000); // 10 loops are in use at the moment, and that is sized at 36,102 bits. - // LOGGER.info("Running main program"); - KafkaController kafkaController = new KafkaController(config); - kafkaController.run(); - } - - // Tests the serialization of the AVRO-file generated in kafkaConsumerTest(). Pathname depends on the configurations set in application.properties file. - // @Test - public void AVROReaderTest() throws IOException { - // Deserialize Users from disk - Config config = new Config(); - Path queueDirectory = Paths.get(config.getQueueDirectory()); - int counter = 0; - for (int i = 1; i<=20; i++) { - File syslogFile = new File( - queueDirectory.toAbsolutePath() - + File.separator - + config.getQueueNamePrefix() - + "." - + i - ); - DatumReader userDatumReader = new SpecificDatumReader<>(SyslogRecord.class); - try (DataFileReader dataFileReader = new DataFileReader<>(syslogFile, userDatumReader)) { - SyslogRecord user = null; - System.out.println(syslogFile.getPath()); - while (dataFileReader.hasNext()) { - user = dataFileReader.next(user); - System.out.println(user); - counter++; + for (int j = 0; j <= 9; j++) { + for (int i = start; i <= end; i++) { + File syslogFile = new File( + queueDirectory.toAbsolutePath() + + File.separator + + "testConsumerTopic" + + j + + "." + + i + ); + try { + boolean result = Files.deleteIfExists(syslogFile.toPath()); //surround it in try catch block + } catch (IOException e) { + throw new RuntimeException(e); } } } - System.out.println("Total number of records: " + counter); - } - - - private static void generateEvents(MockConsumer consumer, String topicName, int partition) { - consumer.addRecord(new ConsumerRecord<>(topicName, - partition, - 0L, - "2022-04-25T07:34:50.804Z".getBytes(StandardCharsets.UTF_8), - "<12>1 2022-04-25T07:34:50.804Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"835bf792-91cf-44e3-976b-518330bb8fd3\" source=\"source\" unixtime=\"1650872090805\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872090\"] [WARN] 2022-04-25 07:34:50,804 com.teragrep.jla_02.Log4j Log - Log4j warn says hi!".getBytes(StandardCharsets.UTF_8) - ) - ); - consumer.addRecord(new ConsumerRecord<>(topicName, - partition, - 1L, - "2022-04-25T07:34:50.806Z".getBytes(StandardCharsets.UTF_8), - "<12>1 2022-04-25T07:34:50.806Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"c3f13f9a-05e2-41bd-b0ad-1eca6fd6fd9a\" source=\"source\" unixtime=\"1650872090806\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872090\"] [ERROR] 2022-04-25 07:34:50,806 com.teragrep.jla_02.Log4j Log - Log4j error says hi!".getBytes(StandardCharsets.UTF_8) - ) - ); - consumer.addRecord(new ConsumerRecord<>(topicName, - partition, - 2L, - "2022-04-25T07:34:50.822Z".getBytes(StandardCharsets.UTF_8), - "<12>1 2022-04-25T07:34:50.822Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02\"][event_id@48577 hostname=\"jla-02\" uuid=\"1848d8a1-2f08-4a1e-bec4-ff9e6dd92553\" source=\"source\" unixtime=\"1650872090822\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872090\"] 470647 [Thread-3] INFO com.teragrep.jla_02.Logback Daily - Logback-daily says hi.".getBytes(StandardCharsets.UTF_8) - ) - ); - - consumer.addRecord(new ConsumerRecord<>(topicName, - partition, - 3L, - "2022-04-25T07:34:50.822Z".getBytes(StandardCharsets.UTF_8), - "<12>1 2022-04-25T07:34:50.822Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02\"][event_id@48577 hostname=\"jla-02\" uuid=\"5e1a0398-c2a0-468d-a562-c3bb31f0f853\" source=\"source\" unixtime=\"1650872090822\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872090\"] 470646 [Thread-3] INFO com.teragrep.jla_02.Logback Audit - Logback-audit says hi.".getBytes(StandardCharsets.UTF_8) - ) - ); - - consumer.addRecord(new ConsumerRecord<>(topicName, - partition, - 4L, - "2022-04-25T07:34:50.822Z".getBytes(StandardCharsets.UTF_8), - "<12>1 2022-04-25T07:34:50.822Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02\"][event_id@48577 hostname=\"jla-02\" uuid=\"6268c3a2-5bda-427f-acce-29416eb445f4\" source=\"source\" unixtime=\"1650872090822\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872090\"] 470647 [Thread-3] INFO com.teragrep.jla_02.Logback Metric - Logback-metric says hi.".getBytes(StandardCharsets.UTF_8) - ) - ); - - consumer.addRecord(new ConsumerRecord<>(topicName, - partition, - 5L, - "2022-04-25T07:34:52.238Z".getBytes(StandardCharsets.UTF_8), - "<12>1 2022-04-25T07:34:52.238Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"b500dcaf-1101-4000-b6b9-bfb052ddbf86\" source=\"source\" unixtime=\"1650872092238\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.238 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info audit says hi!]".getBytes(StandardCharsets.UTF_8) - ) - ); - - consumer.addRecord(new ConsumerRecord<>(topicName, - partition, - 6L, - "2022-04-25T07:34:52.239Z".getBytes(StandardCharsets.UTF_8), - "<12>1 2022-04-25T07:34:52.239Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"05363122-51ac-4c0b-a681-f5868081f56d\" source=\"source\" unixtime=\"1650872092239\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info daily says hi!]".getBytes(StandardCharsets.UTF_8) - ) - ); - - consumer.addRecord(new ConsumerRecord<>(topicName, - partition, - 7L, - "2022-04-25T07:34:52.239Z".getBytes(StandardCharsets.UTF_8), - "<12>1 2022-04-25T07:34:52.239Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"7bbcd843-b795-4c14-b4a1-95f5d445cbcd\" source=\"source\" unixtime=\"1650872092239\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info metric says hi!]".getBytes(StandardCharsets.UTF_8) - ) - ); - - consumer.addRecord(new ConsumerRecord<>(topicName, - partition, - 8L, - "2022-04-25T07:34:52.240Z".getBytes(StandardCharsets.UTF_8), - "<12>1 2022-04-25T07:34:52.240Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"2bc0a9f9-237d-4656-b40a-3038aace37f0\" source=\"source\" unixtime=\"1650872092240\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn audit says hi!]".getBytes(StandardCharsets.UTF_8) - ) - ); - - consumer.addRecord(new ConsumerRecord<>(topicName, - partition, - 9L, - "2022-04-25T07:34:52.240Z".getBytes(StandardCharsets.UTF_8), - "<12>1 2022-04-25T07:34:52.240Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"ecf61e8d-e3a7-48ef-9b73-3c5a5243d2e6\" source=\"source\" unixtime=\"1650872092240\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn daily says hi!]".getBytes(StandardCharsets.UTF_8) - ) - ); - - consumer.addRecord(new ConsumerRecord<>(topicName, - partition, - 10L, - "2022-04-25T07:34:52.241Z".getBytes(StandardCharsets.UTF_8), - "<12>1 2022-04-25T07:34:52.241Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"bf101d5a-e816-4f51-b132-97f8e3431f8e\" source=\"source\" unixtime=\"1650872092241\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.241 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn metric says hi!]".getBytes(StandardCharsets.UTF_8) - ) - ); - - consumer.addRecord(new ConsumerRecord<>(topicName, - partition, - 11L, - "2022-04-25T07:34:52.241Z".getBytes(StandardCharsets.UTF_8), - "<12>1 2022-04-25T07:34:52.241Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"ef94d9e9-3c44-4892-b5a6-bf361d13ff97\" source=\"source\" unixtime=\"1650872092241\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.241 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error audit says hi!]".getBytes(StandardCharsets.UTF_8) - ) - ); - - consumer.addRecord(new ConsumerRecord<>(topicName, - partition, - 12L, - "2022-04-25T07:34:52.242Z".getBytes(StandardCharsets.UTF_8), - "<12>1 2022-04-25T07:34:52.242Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"5bce6e3d-767d-44b4-a044-6c4872f8f2b5\" source=\"source\" unixtime=\"1650872092242\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.242 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error daily says hi!]".getBytes(StandardCharsets.UTF_8) - ) - ); - - consumer.addRecord(new ConsumerRecord<>(topicName, - partition, - 13L, - "2022-04-25T07:34:52.243Z".getBytes(StandardCharsets.UTF_8), - "<12>1 2022-04-25T07:34:52.243Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"3bb55ce4-0ea7-413a-b403-28b174d7ac99\" source=\"source\" unixtime=\"1650872092243\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.243 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error metric says hi!]".getBytes(StandardCharsets.UTF_8) - ) - ); } } From a4f6ed452335d820426bd0512424a87edfa42d91 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Mon, 29 Jan 2024 12:53:15 +0200 Subject: [PATCH 033/146] Improved commenting. Cleaned up code. Cleaned up class names. --- .../consumers/kafka/DatabaseOutput.java | 35 ++++++++----------- .../consumers/kafka/KafkaController.java | 32 +++++------------ ...{KafkaReaderTemp.java => KafkaReader.java} | 10 +++--- ...rdinatorTemp.java => ReadCoordinator.java} | 26 +++++++------- .../consumers/kafka/SyslogAvroWriter.java | 2 +- .../cfe_39/metrics/DurationStatistics.java | 11 +----- 6 files changed, 41 insertions(+), 75 deletions(-) rename src/main/java/com/teragrep/cfe_39/consumers/kafka/{KafkaReaderTemp.java => KafkaReader.java} (89%) rename src/main/java/com/teragrep/cfe_39/consumers/kafka/{ReadCoordinatorTemp.java => ReadCoordinator.java} (66%) diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java index f7dbe083..d5d0b241 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java @@ -6,8 +6,6 @@ import com.teragrep.cfe_39.metrics.topic.TopicCounter; import com.teragrep.cfe_39.metrics.DurationStatistics; import com.teragrep.rlo_06.*; -import org.apache.hadoop.fs.*; -import org.apache.hadoop.hdfs.DistributedFileSystem; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -18,10 +16,9 @@ import java.util.List; import java.util.function.Consumer; -import org.apache.hadoop.conf.Configuration; import java.nio.ByteBuffer; -import java.net.URI; +// TODO: CLEAN COMMENTS, AND ADD COMMENTS WHERE NEEDED. // The kafka stream should first be deserialized using rlo_06 and then serialized again using avro and stored in HDFS. // The target where the record is stored in HDFS is based on the topic, partition and offset. ie. topic_name/0.123456 where offset is 123456 @@ -59,7 +56,7 @@ public class DatabaseOutput implements Consumer> { private final SDVector originHostname; private File syslogFile; private final Config config; - private long approximatedSize; + private long approximatedSize; // FIXME: Not working properly when flush() is not used after append in the AVRO-file. DatabaseOutput( Config config, @@ -72,9 +69,9 @@ public class DatabaseOutput implements Consumer> { this.durationStatistics = durationStatistics; this.topicCounter = topicCounter; this.minimumFreeSpace = 32000000; // TODO: CHECK RIGHT VALUE FOR minimumFreeSpace - this.maximumFileSize = config.getMaximumFileSize();; // Maximum file size should be 64M (64000000). 60800000 is 95% of 64M which is a good approximation point. + this.maximumFileSize = config.getMaximumFileSize();; // Maximum file size should be 64M (64000000). 60800000 is 95% of 64M which should be a good approximation point. - // queueDirectory and queueNamePrefix shouldn't be critical to name according to the HDFS requirements (topic+partition+offset for filename) as it's just used for storing the AVRO-serialized files. + // queueDirectory and queueNamePrefix are only used for temporarily storing the AVRO-serialized files before committing them to HDFS when the file size reaches the threshold (or all records are processed). this.writableQueue = new WritableQueue( config.getQueueDirectory() ); @@ -94,13 +91,13 @@ public class DatabaseOutput implements Consumer> { boolean checkSizeTooLarge(long fileSize, RecordOffsetObject lastObject) { try { - // If the syslogAvroWriter is already initialized, check the filesize so it doesn't go above 64M. + // If the syslogAvroWriter is already initialized, check the filesize so it doesn't go above maximumFileSize. if (fileSize > maximumFileSize) { - // file too large for adding the new record, write the still adequately sized AVRO-file to the HDFS database and create a new empty AVRO-file. + // file too large for adding the new record to it, write the still adequately sized AVRO-file to the HDFS database and create a new empty AVRO-file. // This part closes the writing of now "complete" AVRO-file and stores the file to HDFS. syslogAvroWriter.close(); - // FIXME: UNCOMMENT WHEN DONE TESTING + // FIXME: UNCOMMENT WHEN DONE TESTING THE CONSUMER! This commits the AVRO-file to HDFS. /*try (HDFSWriter writer = new HDFSWriter(config, lastObject)) { writer.commit(syslogFile); // commits the final AVRO-file to HDFS. }*/ @@ -130,9 +127,10 @@ private long rfc3339ToEpoch(ZonedDateTime zonedDateTime) { } // Input parameter is a list of RecordOffsetObjects. Each object contains a record and its metadata (topic, partition and offset). - // Each partition will get their exclusive files. - // The target where the record is stored in HDFS is based on the topic, partition and offset. ie. topic_name/0.123456 where offset is 123456. + // Each partition will get their set of exclusive AVRO-files in HDFS. + // The target where the record is stored in HDFS is based on the topic, partition and last offset. ie. topic_name/0.123456 where last written records offset is 123456. // AVRO-file with a path/name that starts with topic_name/0.X should only contain records from the 0th partition of topic named topic_name, topic_name/1.X should only contain records from 1st partition, etc. + // AVRO-files are created dynamically, thus it is not known which record (and its offset) is written to the file last before committing it to HDFS. The final name (specifically the offset) is decided only when the file is committed to HDFS. @Override public void accept(List recordOffsetObjectList) { long thisTime = Instant.now().toEpochMilli(); @@ -143,11 +141,12 @@ public void accept(List recordOffsetObjectList) { // The recordOffsetObjectList loop will go through all the objects in the list. // While it goes through the list, the contents of the objects are serialized into an AVRO-file. - // When the file size is about to go above 64M, commit the file into HDFS using the latest topic/partition/offset values as the filename and start fresh with an empty AVRO-file. + // When the file size is about to go above 64M, commit the file into HDFS using the latest topic/partition/offset values as the filename and start fresh with a new empty AVRO-file. // Serialize the object that was going to make the file go above 64M into the now empty AVRO-file and continue the loop. // https://pagure.xnet.fi/com-teragrep/rlo_09/blob/avroness/f/src/main/java/com/teragrep/rlo_09/SyslogAvroWriter.java // https://pagure.xnet.fi/com-teragrep/rlo_09/blob/avroness/f/src/main/java/com/teragrep/rlo_09/WriteCoordinator.java // every recordOffsetObject.record on the recordOffsetObjectList basically represents a rlo_09 WriteCoordinator.accept(byte[] bytes) when the list is gone through in a loop. + // TODO: If the prod-environment recordOffsetObjectList ordering is different from what it is in the test environment, add a function that reorders the list based on partition and offset (or better yet, make several AVRO-files that are being used at the same time rather than doing it one AVRO-file at a time as the offset ordering within partitions should always be correct in all scenarios). RecordOffsetObject lastObject = null; long start = Instant.now().toEpochMilli(); // Starts measuring performance here. Measures how long it takes to process the whole recordOffsetObjectList. // This loop goes through all the records of the mock data in a single session. @@ -177,7 +176,7 @@ public void accept(List recordOffsetObjectList) { } else { // Previous partition was fully consumed. Commit file to HDFS and create a new AVRO-file. syslogAvroWriter.close(); - // FIXME: UNCOMMENT WHEN DONE TESTING + // FIXME: UNCOMMENT WHEN DONE TESTING THE CONSUMER! This commits the AVRO-file to HDFS. /*try (HDFSWriter writer = new HDFSWriter(config, lastObject)) { writer.commit(syslogFile); }*/ @@ -251,12 +250,6 @@ public void accept(List recordOffsetObjectList) { approximatedSize += capacity; lastObject = recordOffsetObject; // The difference between actual and approximate file size is about 2,4 % with 64M files. So setting the MaximumFileSize to 95 % of the target should make things work. - - - /*new RFC5424Timestamp(rfc5424Frame.timestamp).toZonedDateTime().toInstant().getEpochSecond(); - rfc5424Frame.appName.toString(); - rfc5424Frame.hostname.toString(); - rfc5424Frame.msg.toString();*/ } } catch (IOException e) { throw new RuntimeException(e); @@ -267,7 +260,7 @@ public void accept(List recordOffsetObjectList) { try { if (syslogAvroWriter != null) { syslogAvroWriter.close(); - // FIXME: UNCOMMENT WHEN DONE TESTING + // FIXME: UNCOMMENT WHEN DONE TESTING THE CONSUMER! This commits the AVRO-file to HDFS. /*try (HDFSWriter writer = new HDFSWriter(config, lastObject)) { writer.commit(syslogFile); // commits the final AVRO-file to HDFS. }*/ diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java index c11d894f..28a57a69 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java @@ -19,11 +19,9 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; -// TODO: Clean up comments. - public class KafkaController { - // rlo_09 KafkaReader is the code that I should take a look at. ReadCoordinator alone won't allow access to the kafka offsets, it must be done in KafkaReader. - // ReadCoordinator uses the KafkaReader, but it's set as private and there are no functions for accessing it through ReadCoordinator. + // ReadCoordinator alone won't allow access to the kafka offsets, it must be done in KafkaReader that is used on rlo_09. + // ReadCoordinator uses the KafkaReader, but it's set as private in rlo_09 and there are no functions for accessing it through ReadCoordinator. // The enable.auto.commit=false is set in config and it is fetched by the config.getKafkaConsumerProperties(). // cfe_30 is already using enable.auto.commit=false, so looking through cfe_30 and rlo_09 ReadCoordinator and KafkaReader functions should get the coding on right track. @@ -59,7 +57,6 @@ public class KafkaController { private boolean keepRunning; private boolean useMockKafkaConsumer; private final int numOfConsumers; - // private final MetricRegistry metricRegistry = new MetricRegistry(); public KafkaController(Config config) { keepRunning = true; @@ -70,7 +67,7 @@ public KafkaController(Config config) { readerKafkaProperties.getProperty("useMockKafkaConsumer", "false") ); if (useMockKafkaConsumer) { - this.kafkaConsumer = MockKafkaConsumerFactoryTemp.getConsumer(0); // Used only for scanning the available topics. + this.kafkaConsumer = MockKafkaConsumerFactoryTemp.getConsumer(0); // A consumer used only for scanning the available topics to be allocated to consumers running in different threads (thus 0 as input parameter). } else { this.kafkaConsumer = new KafkaConsumer<>(config.getKafkaConsumerProperties(), new ByteArrayDeserializer(), new ByteArrayDeserializer()); } @@ -79,7 +76,7 @@ public KafkaController(Config config) { public void run() throws InterruptedException { // register duration statistics - durationStatistics.register(); // FIXED? + durationStatistics.register(); // register per topic counting List topicCounters = new CopyOnWriteArrayList<>(); @@ -97,6 +94,7 @@ public void run() throws InterruptedException { long topicScanDelay = 30000L; Thread.sleep(topicScanDelay); + // For testing purposes only. Stops the run when all the records are consumed from the mockConsumer during test. if (durationStatistics.getTotalRecords() > 0 & useMockKafkaConsumer) { LOGGER.info("Processed all the test records. Closing."); keepRunning = false; @@ -128,7 +126,7 @@ private void createReader(String topic, List listPartitionInfo, L ); // The kafka offsets must be passed to HDFS. The consumer must also be set to manual commits so the HDFS can handle managing the commit offsets within the HDFS filenames. // plain rlo_09.ReadCoordinator won't give access to offset values. Implementing custom rlo_09 code in the package to achieve offset access. - ReadCoordinatorTemp readCoordinator = new ReadCoordinatorTemp( + ReadCoordinator readCoordinator = new ReadCoordinator( topic, config.getKafkaConsumerProperties(), output @@ -144,7 +142,7 @@ private void topicScan(DurationStatistics durationStatistics, List Map> listTopics = kafkaConsumer.listTopics(Duration.ofSeconds(60)); // Topics can be fetched from mock consumer if the consumer has been updated separately with the partition info. Pattern topicsRegex = Pattern.compile(config.getQueueTopicPattern()); // Mock consumer has the partitions in this format: queueTopicPattern=^testConsumerTopic-*$ // Find the topics available in Kafka based on given QueueTopicPattern, both active and in-active. - // Check how partitions are handled, need to allow using consumer groups for partition read assignments. aka. load balancing + // Need to allow using consumer groups for partition read assignments. aka. load balancing Set foundTopics = new HashSet<>(); // 1. Add functionality so the partition information is also fetched for the queried topics. At the moment only the topic names are fetched. @@ -156,7 +154,7 @@ private void topicScan(DurationStatistics durationStatistics, List if (matcher.matches()) { foundTopics.add(entry.getKey()); - // 2. Add functionality so the partition information is also fetched for the queried topics. At the moment only the topic names are fetched. + // 2. Add functionality so the partition information is also fetched for the queried topics. foundPartitions.put(entry.getKey(), entry.getValue()); } @@ -186,20 +184,6 @@ private void topicScan(DurationStatistics durationStatistics, List } }); durationStatistics.report(); - - // Activate all the found in-active topics, in other words create individual consumers for all of them using the createReader()-function. -/* for (String topic : foundTopics) { - LOGGER.info("Activating topic <"+topic+">"); - try { - createReader(topic, topicCounters); - activeTopics.add(topic); - durationStatistics.addAndGetThreads(1); - } - catch (SQLException sqlException) { - LOGGER.error("Topic <"+topic+"> not activated due to reader creation error: " + sqlException); - } - } - durationStatistics.report();*/ } } diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReaderTemp.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReader.java similarity index 89% rename from src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReaderTemp.java rename to src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReader.java index d853aea4..144bd17e 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReaderTemp.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReader.java @@ -8,8 +8,8 @@ import java.time.Duration; import java.util.*; -public class KafkaReaderTemp implements AutoCloseable { - final Logger LOGGER = LoggerFactory.getLogger(KafkaReaderTemp.class); +public class KafkaReader implements AutoCloseable { + final Logger LOGGER = LoggerFactory.getLogger(KafkaReader.class); private Iterator> kafkaRecordsIterator = Collections.emptyIterator(); @@ -17,7 +17,7 @@ public class KafkaReaderTemp implements AutoCloseable { private final java.util.function.Consumer> callbackFunction; - public KafkaReaderTemp( + public KafkaReader( Consumer kafkaConsumer, java.util.function.Consumer> callbackFunction) { this.kafkaConsumer = kafkaConsumer; this.callbackFunction = callbackFunction; @@ -54,10 +54,8 @@ public void read() { // Offset and other required data for HDFS storage are added to the input parameters of the accept() function which processes the consumed record. callbackFunction.accept(recordOffsetObjectList); kafkaConsumer.commitSync(); - /* - commitSync(): - It only commits the offsets that were actually polled and processed. If some offsets were not included in the last poll, then those offsets will not be committed. + commitSync() only commits the offsets that were actually polled and processed. If some offsets were not included in the last poll, then those offsets will not be committed. It will not commit the latest positions for all subscribed partitions. This would interfere with the Consumer Offset management concept of Kafka to be able to re-start an application where it left off. * */ } diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinatorTemp.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinator.java similarity index 66% rename from src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinatorTemp.java rename to src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinator.java index f51fdf95..f5d4c3c1 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinatorTemp.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinator.java @@ -2,24 +2,22 @@ import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.common.serialization.ByteArrayDeserializer; -import org.apache.kafka.common.serialization.ByteArraySerializer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.*; import java.util.function.Consumer; -public class ReadCoordinatorTemp implements Runnable { +public class ReadCoordinator implements Runnable { private static final Logger LOGGER = - LoggerFactory.getLogger(ReadCoordinatorTemp.class); + LoggerFactory.getLogger(ReadCoordinator.class); private final String queueTopic; private final Properties readerKafkaProperties; private final Consumer> callbackFunction; private boolean run = true; - private long offset; - public ReadCoordinatorTemp( + public ReadCoordinator( String queueTopic, Properties readerKafkaProperties, Consumer> callbackFunction) @@ -29,32 +27,33 @@ public ReadCoordinatorTemp( this.callbackFunction = callbackFunction; } - private KafkaReaderTemp createKafkaReader(Properties readerKafkaProperties, + private KafkaReader createKafkaReader(Properties readerKafkaProperties, String topic, Consumer> callbackFunction, boolean useMockKafkaConsumer) { org.apache.kafka.clients.consumer.Consumer kafkaConsumer; - if (useMockKafkaConsumer) { - String name = Thread.currentThread().getName(); + if (useMockKafkaConsumer) { // Test mode is on, create mock consumers with assigned partitions that are not overlapping with each other. + String name = Thread.currentThread().getName(); // Use thread name to identify which thread is running the code. if (Objects.equals(name, "testConsumerTopic1")) { - kafkaConsumer = MockKafkaConsumerFactoryTemp.getConsumer(1); + kafkaConsumer = MockKafkaConsumerFactoryTemp.getConsumer(1); // creates a Kafka MockConsumer that has the odd numbered partitions assigned to it. }else { - kafkaConsumer = MockKafkaConsumerFactoryTemp.getConsumer(2); + kafkaConsumer = MockKafkaConsumerFactoryTemp.getConsumer(2); // creates a Kafka MockConsumer that has the even numbered partitions assigned to it. } - } else { + } else { // Test mode is off, subscribe method should handle assigning the partitions automatically to the consumer based on group id parameters of readerKafkaProperties. kafkaConsumer = new KafkaConsumer<>(readerKafkaProperties, new ByteArrayDeserializer(), new ByteArrayDeserializer()); kafkaConsumer.subscribe(Collections.singletonList(topic)); } - return new KafkaReaderTemp(kafkaConsumer, callbackFunction); + return new KafkaReader(kafkaConsumer, callbackFunction); } + // Part or Runnable implementation, called when the thread is started. @Override public void run() { boolean useMockKafkaConsumer = Boolean.parseBoolean(readerKafkaProperties.getProperty("useMockKafkaConsumer", "false")); try ( - KafkaReaderTemp kafkaReader = createKafkaReader( + KafkaReader kafkaReader = createKafkaReader( readerKafkaProperties, queueTopic, callbackFunction, @@ -67,6 +66,7 @@ public void run() { } } + // remove? public void stop() { run = false; } diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java index d08defb2..a97a95ea 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java @@ -47,7 +47,7 @@ class SyslogAvroWriter implements AutoCloseable { void write(SyslogRecord syslogRecord) throws IOException{ dataFileWriter.append(syslogRecord); - dataFileWriter.flush(); // getFileSize() doesn't work properly if dataFileWriter.flush() is not called after appending a new record to the AVRO-file. + dataFileWriter.flush(); // FIXME: getFileSize() doesn't work properly if dataFileWriter.flush() is not called after appending a new record to the AVRO-file. // Avro files 'flush' must be called as few times as possible. Check memory usage impact. Use only automatic flush which is triggered when .close() is called. // To use the automatic flush AND have a working getFileSize(), the file size must be tracked separately. Approximate the file size by adding the original file size before any appending to the sum of record sizes. diff --git a/src/main/java/com/teragrep/cfe_39/metrics/DurationStatistics.java b/src/main/java/com/teragrep/cfe_39/metrics/DurationStatistics.java index e94518ec..e4ed4b43 100644 --- a/src/main/java/com/teragrep/cfe_39/metrics/DurationStatistics.java +++ b/src/main/java/com/teragrep/cfe_39/metrics/DurationStatistics.java @@ -3,19 +3,13 @@ import com.codahale.metrics.Counter; import com.codahale.metrics.Meter; import com.codahale.metrics.MetricRegistry; -import com.codahale.metrics.Timer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import javax.management.DynamicMBean; import java.time.Instant; -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.TimeUnit; -// TODO: Change DurationStatistics to use dropwizard instead of mxj_01. public class DurationStatistics { - MetricRegistry metricRegistry = new MetricRegistry(); // TODO: MetricRegistry is initialized here. Implement the different mxj_01 metrics objects to it. + MetricRegistry metricRegistry = new MetricRegistry(); private static final Logger LOGGER = LoggerFactory.getLogger(DurationStatistics.class); private Instant lastReportTime = Instant.now(); private long lastBytes = 0L; @@ -90,18 +84,15 @@ public void log() { public long addAndGetThreads(long delta) { threadsStat.mark(delta); return threadsStat.getCount(); - // return threadsStat.getAtomicLong().addAndGet(delta); } public long addAndGetBytes(long delta) { bytesStat.mark(delta); return bytesStat.getCount(); - // return bytesStat.getAtomicLong().addAndGet(delta); } public long addAndGetRecords(long delta) { recordsStat.mark(delta); return recordsStat.getCount(); - // return recordsStat.getAtomicLong().addAndGet(delta); } } From 884850084978ba696c9f54bd39edf1906a5869e6 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Tue, 30 Jan 2024 11:14:07 +0200 Subject: [PATCH 034/146] Added licenses to files. Removed obsolete TopicStatistics.java. --- src/main/java/com/teragrep/cfe_39/Config.java | 17 ++++++ .../consumers/kafka/DatabaseOutput.java | 17 ++++++ .../cfe_39/consumers/kafka/HDFSWriter.java | 18 +++++- .../consumers/kafka/KafkaController.java | 22 +++++-- .../cfe_39/consumers/kafka/KafkaReader.java | 17 ++++++ .../kafka/MockKafkaConsumerFactoryTemp.java | 17 ++++++ .../consumers/kafka/ReadCoordinator.java | 17 ++++++ .../consumers/kafka/RecordOffsetObject.java | 17 ++++++ .../consumers/kafka/SyslogAvroWriter.java | 17 ++++++ .../consumers/kafka/queue/QueueUtilities.java | 17 ++++++ .../consumers/kafka/queue/WritableQueue.java | 17 ++++++ .../cfe_39/metrics/DurationStatistics.java | 17 ++++++ .../cfe_39/metrics/topic/TopicCounter.java | 17 ++++++ .../cfe_39/metrics/topic/TopicStatistics.java | 60 ------------------- 14 files changed, 221 insertions(+), 66 deletions(-) delete mode 100644 src/main/java/com/teragrep/cfe_39/metrics/topic/TopicStatistics.java diff --git a/src/main/java/com/teragrep/cfe_39/Config.java b/src/main/java/com/teragrep/cfe_39/Config.java index f4c9df4a..989be060 100644 --- a/src/main/java/com/teragrep/cfe_39/Config.java +++ b/src/main/java/com/teragrep/cfe_39/Config.java @@ -1,3 +1,20 @@ +/* + HDFS Data Ingestion for PTH_06 use CFE-39 + Copyright (C) 2022 Fail-Safe IT Solutions Oy + + 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. + */ + package com.teragrep.cfe_39; import org.apache.logging.log4j.core.config.ConfigurationSource; diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java index d5d0b241..000d7c6b 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java @@ -1,3 +1,20 @@ +/* + HDFS Data Ingestion for PTH_06 use CFE-39 + Copyright (C) 2022 Fail-Safe IT Solutions Oy + + 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. + */ + package com.teragrep.cfe_39.consumers.kafka; import com.teragrep.cfe_39.Config; diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java index e0d49585..677cb811 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java @@ -1,3 +1,20 @@ +/* + HDFS Data Ingestion for PTH_06 use CFE-39 + Copyright (C) 2022 Fail-Safe IT Solutions Oy + + 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. + */ + package com.teragrep.cfe_39.consumers.kafka; import com.teragrep.cfe_39.Config; @@ -7,7 +24,6 @@ import org.apache.hadoop.security.UserGroupInformation; import java.io.File; -import java.io.FileInputStream; import java.io.IOException; import java.net.URI; import java.nio.file.Files; diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java index 28a57a69..dc1c435d 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java @@ -1,9 +1,25 @@ +/* + HDFS Data Ingestion for PTH_06 use CFE-39 + Copyright (C) 2022 Fail-Safe IT Solutions Oy + + 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. + */ + package com.teragrep.cfe_39.consumers.kafka; import com.teragrep.cfe_39.Config; import com.teragrep.cfe_39.metrics.*; import com.teragrep.cfe_39.metrics.topic.TopicCounter; -import com.teragrep.cfe_39.metrics.topic.TopicStatistics; import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.serialization.ByteArrayDeserializer; @@ -80,10 +96,6 @@ public void run() throws InterruptedException { // register per topic counting List topicCounters = new CopyOnWriteArrayList<>(); - TopicStatistics topicMetrics = new TopicStatistics( - topicCounters - ); - topicMetrics.register(); while (keepRunning) { LOGGER.debug("Scanning for threads"); diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReader.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReader.java index 144bd17e..d4e6674d 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReader.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReader.java @@ -1,3 +1,20 @@ +/* + HDFS Data Ingestion for PTH_06 use CFE-39 + Copyright (C) 2022 Fail-Safe IT Solutions Oy + + 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. + */ + package com.teragrep.cfe_39.consumers.kafka; import org.apache.kafka.clients.consumer.*; diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java index f251ead3..8c511f58 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java @@ -1,3 +1,20 @@ +/* + HDFS Data Ingestion for PTH_06 use CFE-39 + Copyright (C) 2022 Fail-Safe IT Solutions Oy + + 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. + */ + package com.teragrep.cfe_39.consumers.kafka; import org.apache.kafka.clients.consumer.Consumer; diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinator.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinator.java index f5d4c3c1..156bcf6a 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinator.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinator.java @@ -1,3 +1,20 @@ +/* + HDFS Data Ingestion for PTH_06 use CFE-39 + Copyright (C) 2022 Fail-Safe IT Solutions Oy + + 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. + */ + package com.teragrep.cfe_39.consumers.kafka; import org.apache.kafka.clients.consumer.KafkaConsumer; diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/RecordOffsetObject.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/RecordOffsetObject.java index a7b27f3e..b0516965 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/RecordOffsetObject.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/RecordOffsetObject.java @@ -1,3 +1,20 @@ +/* + HDFS Data Ingestion for PTH_06 use CFE-39 + Copyright (C) 2022 Fail-Safe IT Solutions Oy + + 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. + */ + package com.teragrep.cfe_39.consumers.kafka; // This is the class for handling the Kafka record topic/partition/offset data that are required for HDFS storage. diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java index a97a95ea..4eeb290c 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java @@ -1,3 +1,20 @@ +/* + HDFS Data Ingestion for PTH_06 use CFE-39 + Copyright (C) 2022 Fail-Safe IT Solutions Oy + + 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. + */ + package com.teragrep.cfe_39.consumers.kafka; import com.teragrep.cfe_39.avro.SyslogRecord; diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/queue/QueueUtilities.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/queue/QueueUtilities.java index 3501038c..46873bd4 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/queue/QueueUtilities.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/queue/QueueUtilities.java @@ -1,3 +1,20 @@ +/* + HDFS Data Ingestion for PTH_06 use CFE-39 + Copyright (C) 2022 Fail-Safe IT Solutions Oy + + 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. + */ + package com.teragrep.cfe_39.consumers.kafka.queue; import org.slf4j.Logger; diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/queue/WritableQueue.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/queue/WritableQueue.java index fa829c02..80618adf 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/queue/WritableQueue.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/queue/WritableQueue.java @@ -1,3 +1,20 @@ +/* + HDFS Data Ingestion for PTH_06 use CFE-39 + Copyright (C) 2022 Fail-Safe IT Solutions Oy + + 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. + */ + package com.teragrep.cfe_39.consumers.kafka.queue; import java.io.File; diff --git a/src/main/java/com/teragrep/cfe_39/metrics/DurationStatistics.java b/src/main/java/com/teragrep/cfe_39/metrics/DurationStatistics.java index e4ed4b43..d33b8b1b 100644 --- a/src/main/java/com/teragrep/cfe_39/metrics/DurationStatistics.java +++ b/src/main/java/com/teragrep/cfe_39/metrics/DurationStatistics.java @@ -1,3 +1,20 @@ +/* + HDFS Data Ingestion for PTH_06 use CFE-39 + Copyright (C) 2022 Fail-Safe IT Solutions Oy + + 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. + */ + package com.teragrep.cfe_39.metrics; import com.codahale.metrics.Counter; diff --git a/src/main/java/com/teragrep/cfe_39/metrics/topic/TopicCounter.java b/src/main/java/com/teragrep/cfe_39/metrics/topic/TopicCounter.java index 828eadf5..7212045c 100644 --- a/src/main/java/com/teragrep/cfe_39/metrics/topic/TopicCounter.java +++ b/src/main/java/com/teragrep/cfe_39/metrics/topic/TopicCounter.java @@ -1,3 +1,20 @@ +/* + HDFS Data Ingestion for PTH_06 use CFE-39 + Copyright (C) 2022 Fail-Safe IT Solutions Oy + + 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. + */ + package com.teragrep.cfe_39.metrics.topic; diff --git a/src/main/java/com/teragrep/cfe_39/metrics/topic/TopicStatistics.java b/src/main/java/com/teragrep/cfe_39/metrics/topic/TopicStatistics.java deleted file mode 100644 index 3a0f2788..00000000 --- a/src/main/java/com/teragrep/cfe_39/metrics/topic/TopicStatistics.java +++ /dev/null @@ -1,60 +0,0 @@ -package com.teragrep.cfe_39.metrics.topic; - -// TODO: Add dropwizard metrics implementation to replace old mxj_01 implementation. - -import javax.management.*; -import java.util.List; - -public class TopicStatistics { - private final List topicList; - - // FIXME - // private final DynamicBean dynamicBean; - - public TopicStatistics(List topicList) { - this.topicList = topicList; - - // FIXME -/* // page - CompositeDataWriter topicCounterWriter = CompositeDataWriter.builder(TopicCounter.class) - .withTypeName("topic") - .withTypeDescription("Topic throughput") - .withSimpleAttribute("topicName", TopicCounter::getTopicName) - .withSimpleAttribute("totalRecords", TopicCounter::getTotalRecords) - .withSimpleAttribute("totalBytes", TopicCounter::getTotalBytes) - .withSimpleAttribute("recordsPerSecond", TopicCounter::getRecordsPerSecond) - .withSimpleAttribute("bytesPerSecond", TopicCounter::getBytesPerSecond) - .withSimpleAttribute("kafkaLatency", TopicCounter::getKafkaLatency) - .withSimpleAttribute("databaseLatency", TopicCounter::getDatabaseLatency) - .build(); - - // book - TabularDataWriter topicListWriter = TabularDataWriter.builder(TopicCounter.class) - .withTypeName("topics") - .withTypeDescription("Topics counted") - .withIndexName("topicName") - .withCompositeDataWriter(topicCounterWriter) - .build(); - - - this.dynamicBean = DynamicBean.builder() - .withTabularAttribute( - "TopicStatistics", - () -> topicList, - topicListWriter - ) - .build();*/ - } - - public DynamicMBean register() { - // FIXME - /*MXJEndpoint mxjEndpoint = new MXJEndpoint( - "com.teragrep.cfe_30", - "Metrics", - "Topic", - dynamicBean - ); - return mxjEndpoint.register();*/ - return null; - } -} From bb6694532153a72ee3fbecc905b50aef2ff3732a Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Tue, 30 Jan 2024 15:47:32 +0200 Subject: [PATCH 035/146] Replaced rlo_09 dependency with Kafka-clients. --- pom.xml | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/pom.xml b/pom.xml index ea4d2be1..679665df 100644 --- a/pom.xml +++ b/pom.xml @@ -108,10 +108,11 @@ rlo_06 9.0.1 + - com.teragrep - rlo_09 - 2.0.4 + org.apache.kafka + kafka-clients + 3.4.0 From 1713376ab25b7274affb848b3cddf90f3dc79ee4 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Thu, 1 Feb 2024 16:06:53 +0200 Subject: [PATCH 036/146] Setting up tests for HDFS write. --- .../java/com/teragrep/cfe_39/HdfsTest.java | 208 ++++++++++++++++++ .../teragrep/cfe_39/KafkaConsumerTest.java | 88 ++++---- 2 files changed, 252 insertions(+), 44 deletions(-) create mode 100644 src/test/java/com/teragrep/cfe_39/HdfsTest.java diff --git a/src/test/java/com/teragrep/cfe_39/HdfsTest.java b/src/test/java/com/teragrep/cfe_39/HdfsTest.java new file mode 100644 index 00000000..c57cf86d --- /dev/null +++ b/src/test/java/com/teragrep/cfe_39/HdfsTest.java @@ -0,0 +1,208 @@ +package com.teragrep.cfe_39; + +import com.teragrep.cfe_39.avro.SyslogRecord; +import com.teragrep.cfe_39.consumers.kafka.KafkaController; +import com.teragrep.cfe_39.consumers.kafka.RecordOffsetObject; +import org.apache.avro.file.DataFileReader; +import org.apache.avro.file.DataFileStream; +import org.apache.avro.io.DatumReader; +import org.apache.avro.specific.SpecificDatumReader; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +import java.io.File; +import java.io.IOException; +import java.net.URI; +import java.nio.file.Files; + +public class HdfsTest { + + // Generate AVRO-files for testing the HDFS writes. + @BeforeAll + public static void generateTestData() throws IOException, InterruptedException { + Config config = new Config(); + config.setMaximumFileSize(3000); // 10 loops (140 records) are in use at the moment, and that is sized at 36,102 bits. + KafkaController kafkaController = new KafkaController(config); + kafkaController.run(); + } + + // Delete the generated AVRO-files. + @AfterAll + public static void deleteTestData() throws IOException { + Config config = new Config(); + Path queueDirectory = new Path(config.getQueueDirectory()); // Paths.get(config.getQueueDirectory()); + for (int j = 0; j <= 9; j++) { + for (int i = 1; i <= 5; i++) { + File syslogFile = new File( + queueDirectory.toUri() + + File.separator + + "testConsumerTopic" + + j + + "." + + i + ); + try { + boolean result = Files.deleteIfExists(syslogFile.toPath()); //surround it in try catch block + } catch (IOException e) { + throw new RuntimeException(e); + } + } + } + } + + @Test + public void hdfsWriteTest() { + Config config = null; + try { + config = new Config(); + } catch (IOException e){ + System.out.println("Can't load config: " + e); + System.exit(1); + } catch (IllegalArgumentException e) { + System.out.println("Got invalid config: " + e); + System.exit(1); + } + + Path queueDirectory = new Path(config.getQueueDirectory()); + for (int j = 0; j <= 9; j++) { + for (int i = 1; i <= 2; i++) { + File syslogFile = new File( + queueDirectory.toUri() + + File.separator + + "testConsumerTopic" + + j + + "." + + i + ); + + // generate lastObject from the last record in the file in this test + DatumReader userDatumReader = new SpecificDatumReader<>(SyslogRecord.class); + SyslogRecord lastRecord = null; + try (DataFileReader dataFileReader = new DataFileReader<>(syslogFile, userDatumReader)) { + while (dataFileReader.hasNext()) { + lastRecord = dataFileReader.next(lastRecord); + } + } catch (IOException e) { + throw new RuntimeException(e); + } + + assert lastRecord != null; + RecordOffsetObject lastObject = new RecordOffsetObject("testConsumerTopic", Integer.parseInt(lastRecord.getPartition().toString()), lastRecord.getOffset(), null); // Fetch input parameters from the lastRecord SyslogRecord-object. + System.out.println("Last record in the " + syslogFile.getName() + " file:" + "\ntopic: " + lastObject.topic + "\npartition: " + lastObject.partition + "\noffset: " + lastObject.offset + "\n"); + /*try (HDFSWriter writer = new HDFSWriter(config, lastObject)) { + writer.commit(syslogFile); // commits the final AVRO-file to HDFS. + } catch (IOException e) { + throw new RuntimeException(e); + }*/ + + // Check that the file was stored to HDFS properly. + try { + wait(1000); + hdfsReadCheck("testConsumerTopic", Integer.parseInt(lastRecord.getPartition().toString()), lastRecord.getOffset()); + } catch (InterruptedException | IOException e) { + throw new RuntimeException(e); + } + + } + } + } + + public void hdfsReadCheck(String testConsumerTopic, int partition, long offset) throws IOException { + + Config config = null; + try { + config = new Config(); + } catch (IOException e){ + System.out.println("Can't load config: " + e); + System.exit(1); + } catch (IllegalArgumentException e) { + System.out.println("Got invalid config: " + e); + System.exit(1); + } + + // Check that the files were properly written to HDFS with a read test. + String hdfsuri = config.getHdfsuri(); + + String path = config.getHdfsPath()+"/"+testConsumerTopic; + String fileName = partition+"."+offset; + // ====== Init HDFS File System Object + Configuration conf = new Configuration(); + // Set FileSystem URI + conf.set("fs.defaultFS", hdfsuri); + // Because of Maven + conf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); + conf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); + // Set HADOOP user + System.setProperty("HADOOP_USER_NAME", "hdfs"); + System.setProperty("hadoop.home.dir", "/"); + //Get the filesystem - HDFS + FileSystem fs = FileSystem.get(URI.create(hdfsuri), conf); + + //==== Create folder if not exists + Path workingDir=fs.getWorkingDirectory(); + Path newFolderPath= new Path(path); + if(!fs.exists(newFolderPath)) { + // Create new Directory + fs.mkdirs(newFolderPath); + // logger.info("Path "+path+" created."); + } + + // This is the HDFS write path for the files: + // Path hdfswritepath = new Path(newFolderPath + "/" + fileName); where newFolderPath is config.getHdfsPath() + "/" + lastObject.topic; and filename is lastObject.partition+"."+lastObject.offset; + + //==== Read files + // logger.info("Read file into hdfs"); + //Create a path + Path hdfsreadpath = new Path(newFolderPath + "/" + fileName); // The path should be the same that was used in writing the file to HDFS. + //Init input stream + FSDataInputStream inputStream = fs.open(hdfsreadpath); + //The data is in AVRO-format, so it can't be read as a string. + DataFileStream reader = new DataFileStream<>(inputStream, new SpecificDatumReader<>(SyslogRecord.class)); + SyslogRecord record = null; + while (reader.hasNext()) { + reader.next(); + record = reader.next(record); + System.out.println(record); + // Assert records here like it is done in KafkaConsumerTest.avroReader(). + if (offset <= 0) { + Assertions.assertEquals("{\"timestamp\": 1650872090804000, \"message\": \"[WARN] 2022-04-25 07:34:50,804 com.teragrep.jla_02.Log4j Log - Log4j warn says hi!\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partition + "\", \"offset\": 0, \"origin\": \"jla-02.default\"}", record.toString()); + } else if (offset == 1) { + Assertions.assertEquals("{\"timestamp\": 1650872090806000, \"message\": \"[ERROR] 2022-04-25 07:34:50,806 com.teragrep.jla_02.Log4j Log - Log4j error says hi!\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partition + "\", \"offset\": 1, \"origin\": \"jla-02.default\"}", record.toString()); + } else if (offset == 2) { + Assertions.assertEquals("{\"timestamp\": 1650872090822000, \"message\": \"470647 [Thread-3] INFO com.teragrep.jla_02.Logback Daily - Logback-daily says hi.\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partition + "\", \"offset\": 2, \"origin\": \"jla-02\"}", record.toString()); + } else if (offset == 3) { + Assertions.assertEquals("{\"timestamp\": 1650872090822000, \"message\": \"470646 [Thread-3] INFO com.teragrep.jla_02.Logback Audit - Logback-audit says hi.\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partition + "\", \"offset\": 3, \"origin\": \"jla-02\"}", record.toString()); + } else if (offset == 4) { + Assertions.assertEquals("{\"timestamp\": 1650872090822000, \"message\": \"470647 [Thread-3] INFO com.teragrep.jla_02.Logback Metric - Logback-metric says hi.\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partition + "\", \"offset\": 4, \"origin\": \"jla-02\"}", record.toString()); + } else if (offset == 5) { + Assertions.assertEquals("{\"timestamp\": 1650872092238000, \"message\": \"25.04.2022 07:34:52.238 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info audit says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partition + "\", \"offset\": 5, \"origin\": \"jla-02.default\"}", record.toString()); + } else if (offset == 6) { + Assertions.assertEquals("{\"timestamp\": 1650872092239000, \"message\": \"25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info daily says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partition + "\", \"offset\": 6, \"origin\": \"jla-02.default\"}", record.toString()); + } else if (offset == 7) { + Assertions.assertEquals("{\"timestamp\": 1650872092239000, \"message\": \"25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info metric says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partition + "\", \"offset\": 7, \"origin\": \"jla-02.default\"}", record.toString()); + } else if (offset == 8) { + Assertions.assertEquals("{\"timestamp\": 1650872092240000, \"message\": \"25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn audit says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partition + "\", \"offset\": 8, \"origin\": \"jla-02.default\"}", record.toString()); + } else if (offset == 9) { + Assertions.assertEquals("{\"timestamp\": 1650872092240000, \"message\": \"25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn daily says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partition + "\", \"offset\": 9, \"origin\": \"jla-02.default\"}", record.toString()); + } else if (offset == 10) { + Assertions.assertEquals("{\"timestamp\": 1650872092241000, \"message\": \"25.04.2022 07:34:52.241 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn metric says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partition + "\", \"offset\": 10, \"origin\": \"jla-02.default\"}", record.toString()); + } else if (offset == 11) { + Assertions.assertEquals("{\"timestamp\": 1650872092241000, \"message\": \"25.04.2022 07:34:52.241 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error audit says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partition + "\", \"offset\": 11, \"origin\": \"jla-02.default\"}", record.toString()); + } else if (offset == 12) { + Assertions.assertEquals("{\"timestamp\": 1650872092242000, \"message\": \"25.04.2022 07:34:52.242 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error daily says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partition + "\", \"offset\": 12, \"origin\": \"jla-02.default\"}", record.toString()); + } else { + Assertions.assertEquals("{\"timestamp\": 1650872092243000, \"message\": \"25.04.2022 07:34:52.243 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error metric says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partition + "\", \"offset\": 13, \"origin\": \"jla-02.default\"}", record.toString()); + } + + } + // logger.info(out); + inputStream.close(); + fs.close(); + } +} diff --git a/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java b/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java index fbb1672c..5d13c7d2 100644 --- a/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java +++ b/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java @@ -95,50 +95,50 @@ public int avroReader(int start, int end) throws IOException { System.out.println(user); counter++; // All the mock data is generated from a set of 14 records. - if (looper <= 0) { - Assertions.assertEquals("{\"timestamp\": 1650872090804000, \"message\": \"[WARN] 2022-04-25 07:34:50,804 com.teragrep.jla_02.Log4j Log - Log4j warn says hi!\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \""+partitionCounter+"\", \"offset\": 0, \"origin\": \"jla-02.default\"}", user.toString()); - looper++; - } else if (looper == 1) { - Assertions.assertEquals("{\"timestamp\": 1650872090806000, \"message\": \"[ERROR] 2022-04-25 07:34:50,806 com.teragrep.jla_02.Log4j Log - Log4j error says hi!\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \""+partitionCounter+"\", \"offset\": 1, \"origin\": \"jla-02.default\"}", user.toString()); - looper++; - } else if (looper == 2) { - Assertions.assertEquals("{\"timestamp\": 1650872090822000, \"message\": \"470647 [Thread-3] INFO com.teragrep.jla_02.Logback Daily - Logback-daily says hi.\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \""+partitionCounter+"\", \"offset\": 2, \"origin\": \"jla-02\"}", user.toString()); - looper++; - } else if (looper == 3) { - Assertions.assertEquals("{\"timestamp\": 1650872090822000, \"message\": \"470646 [Thread-3] INFO com.teragrep.jla_02.Logback Audit - Logback-audit says hi.\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \""+partitionCounter+"\", \"offset\": 3, \"origin\": \"jla-02\"}", user.toString()); - looper++; - } else if (looper == 4) { - Assertions.assertEquals("{\"timestamp\": 1650872090822000, \"message\": \"470647 [Thread-3] INFO com.teragrep.jla_02.Logback Metric - Logback-metric says hi.\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \""+partitionCounter+"\", \"offset\": 4, \"origin\": \"jla-02\"}", user.toString()); - looper++; - } else if (looper == 5) { - Assertions.assertEquals("{\"timestamp\": 1650872092238000, \"message\": \"25.04.2022 07:34:52.238 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info audit says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \""+partitionCounter+"\", \"offset\": 5, \"origin\": \"jla-02.default\"}", user.toString()); - looper++; - } else if (looper == 6) { - Assertions.assertEquals("{\"timestamp\": 1650872092239000, \"message\": \"25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info daily says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \""+partitionCounter+"\", \"offset\": 6, \"origin\": \"jla-02.default\"}", user.toString()); - looper++; - } else if (looper == 7) { - Assertions.assertEquals("{\"timestamp\": 1650872092239000, \"message\": \"25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info metric says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \""+partitionCounter+"\", \"offset\": 7, \"origin\": \"jla-02.default\"}", user.toString()); - looper++; - } else if (looper == 8) { - Assertions.assertEquals("{\"timestamp\": 1650872092240000, \"message\": \"25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn audit says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \""+partitionCounter+"\", \"offset\": 8, \"origin\": \"jla-02.default\"}", user.toString()); - looper++; - } else if (looper == 9) { - Assertions.assertEquals("{\"timestamp\": 1650872092240000, \"message\": \"25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn daily says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \""+partitionCounter+"\", \"offset\": 9, \"origin\": \"jla-02.default\"}", user.toString()); - looper++; - } else if (looper == 10) { - Assertions.assertEquals("{\"timestamp\": 1650872092241000, \"message\": \"25.04.2022 07:34:52.241 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn metric says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \""+partitionCounter+"\", \"offset\": 10, \"origin\": \"jla-02.default\"}", user.toString()); - looper++; - } else if (looper == 11) { - Assertions.assertEquals("{\"timestamp\": 1650872092241000, \"message\": \"25.04.2022 07:34:52.241 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error audit says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \""+partitionCounter+"\", \"offset\": 11, \"origin\": \"jla-02.default\"}", user.toString()); - looper++; - } else if (looper == 12) { - Assertions.assertEquals("{\"timestamp\": 1650872092242000, \"message\": \"25.04.2022 07:34:52.242 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error daily says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \""+partitionCounter+"\", \"offset\": 12, \"origin\": \"jla-02.default\"}", user.toString()); - looper++; - } else { - Assertions.assertEquals("{\"timestamp\": 1650872092243000, \"message\": \"25.04.2022 07:34:52.243 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error metric says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \""+partitionCounter+"\", \"offset\": 13, \"origin\": \"jla-02.default\"}", user.toString()); - looper = 0; - partitionCounter++; - } + if (looper <= 0) { + Assertions.assertEquals("{\"timestamp\": 1650872090804000, \"message\": \"[WARN] 2022-04-25 07:34:50,804 com.teragrep.jla_02.Log4j Log - Log4j warn says hi!\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 0, \"origin\": \"jla-02.default\"}", user.toString()); + looper++; + } else if (looper == 1) { + Assertions.assertEquals("{\"timestamp\": 1650872090806000, \"message\": \"[ERROR] 2022-04-25 07:34:50,806 com.teragrep.jla_02.Log4j Log - Log4j error says hi!\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 1, \"origin\": \"jla-02.default\"}", user.toString()); + looper++; + } else if (looper == 2) { + Assertions.assertEquals("{\"timestamp\": 1650872090822000, \"message\": \"470647 [Thread-3] INFO com.teragrep.jla_02.Logback Daily - Logback-daily says hi.\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 2, \"origin\": \"jla-02\"}", user.toString()); + looper++; + } else if (looper == 3) { + Assertions.assertEquals("{\"timestamp\": 1650872090822000, \"message\": \"470646 [Thread-3] INFO com.teragrep.jla_02.Logback Audit - Logback-audit says hi.\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 3, \"origin\": \"jla-02\"}", user.toString()); + looper++; + } else if (looper == 4) { + Assertions.assertEquals("{\"timestamp\": 1650872090822000, \"message\": \"470647 [Thread-3] INFO com.teragrep.jla_02.Logback Metric - Logback-metric says hi.\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 4, \"origin\": \"jla-02\"}", user.toString()); + looper++; + } else if (looper == 5) { + Assertions.assertEquals("{\"timestamp\": 1650872092238000, \"message\": \"25.04.2022 07:34:52.238 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info audit says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 5, \"origin\": \"jla-02.default\"}", user.toString()); + looper++; + } else if (looper == 6) { + Assertions.assertEquals("{\"timestamp\": 1650872092239000, \"message\": \"25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info daily says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 6, \"origin\": \"jla-02.default\"}", user.toString()); + looper++; + } else if (looper == 7) { + Assertions.assertEquals("{\"timestamp\": 1650872092239000, \"message\": \"25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info metric says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 7, \"origin\": \"jla-02.default\"}", user.toString()); + looper++; + } else if (looper == 8) { + Assertions.assertEquals("{\"timestamp\": 1650872092240000, \"message\": \"25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn audit says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 8, \"origin\": \"jla-02.default\"}", user.toString()); + looper++; + } else if (looper == 9) { + Assertions.assertEquals("{\"timestamp\": 1650872092240000, \"message\": \"25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn daily says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 9, \"origin\": \"jla-02.default\"}", user.toString()); + looper++; + } else if (looper == 10) { + Assertions.assertEquals("{\"timestamp\": 1650872092241000, \"message\": \"25.04.2022 07:34:52.241 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn metric says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 10, \"origin\": \"jla-02.default\"}", user.toString()); + looper++; + } else if (looper == 11) { + Assertions.assertEquals("{\"timestamp\": 1650872092241000, \"message\": \"25.04.2022 07:34:52.241 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error audit says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 11, \"origin\": \"jla-02.default\"}", user.toString()); + looper++; + } else if (looper == 12) { + Assertions.assertEquals("{\"timestamp\": 1650872092242000, \"message\": \"25.04.2022 07:34:52.242 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error daily says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 12, \"origin\": \"jla-02.default\"}", user.toString()); + looper++; + } else { + Assertions.assertEquals("{\"timestamp\": 1650872092243000, \"message\": \"25.04.2022 07:34:52.243 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error metric says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 13, \"origin\": \"jla-02.default\"}", user.toString()); + looper = 0; + partitionCounter++; + } } } } From 6198f30f952ea8004b953e57f8ef266a8e1ff008 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Mon, 5 Feb 2024 16:43:49 +0200 Subject: [PATCH 037/146] Setting up MiniCluster for HDFS testing. Debugging issues in HDFS tests. --- pom.xml | 32 ++++- src/main/java/com/teragrep/cfe_39/Config.java | 7 +- .../java/com/teragrep/cfe_39/HdfsTest.java | 130 ++++++++++++------ .../teragrep/cfe_39/KafkaConsumerTest.java | 2 +- 4 files changed, 126 insertions(+), 45 deletions(-) diff --git a/pom.xml b/pom.xml index 679665df..a28b7920 100644 --- a/pom.xml +++ b/pom.xml @@ -131,6 +131,36 @@ hadoop-client 3.3.6 + + + org.apache.hadoop + hadoop-minicluster + 3.3.6 + + + + + org.apache.hadoop + hadoop-common + 3.3.6 + + + + org.apache.hadoop + hadoop-hdfs + 3.3.6 + test + + + + org.mockito + mockito-core + 4.11.0 + + + + + org.apache.logging.log4j log4j-slf4j2-impl @@ -139,7 +169,7 @@ org.apache.logging.log4j log4j-core - 2.19.0 + 2.20.0 org.slf4j diff --git a/src/main/java/com/teragrep/cfe_39/Config.java b/src/main/java/com/teragrep/cfe_39/Config.java index 989be060..182da80a 100644 --- a/src/main/java/com/teragrep/cfe_39/Config.java +++ b/src/main/java/com/teragrep/cfe_39/Config.java @@ -51,7 +51,7 @@ public class Config { private final boolean overrideTableLocation; private final String tableLocation; private final String hdfsPath; - private final String hdfsuri; + private String hdfsuri; private final String queueDirectory; private final String queueNamePrefix; private final String kerberosHost; @@ -112,7 +112,7 @@ public class Config { // HDFS this.hdfsPath = properties.getProperty("hdfsPath", "hdfs:///opt/teragrep/cfe_39/srv/"); - this.hdfsuri = properties.getProperty("hdfsuri", ""); + this.hdfsuri = properties.getProperty("hdfsuri", "hdfs://localhost:45937/"); // AVRO this.queueDirectory = properties.getProperty("queueDirectory", ""); @@ -188,6 +188,9 @@ public String getDbPassword() { public String getHdfsPath() { return hdfsPath; } + public void setHdfsuri(String input) { + this.hdfsuri = input; + } public String getHdfsuri() { return hdfsuri; } diff --git a/src/test/java/com/teragrep/cfe_39/HdfsTest.java b/src/test/java/com/teragrep/cfe_39/HdfsTest.java index c57cf86d..08402636 100644 --- a/src/test/java/com/teragrep/cfe_39/HdfsTest.java +++ b/src/test/java/com/teragrep/cfe_39/HdfsTest.java @@ -1,6 +1,7 @@ package com.teragrep.cfe_39; import com.teragrep.cfe_39.avro.SyslogRecord; +import com.teragrep.cfe_39.consumers.kafka.HDFSWriter; import com.teragrep.cfe_39.consumers.kafka.KafkaController; import com.teragrep.cfe_39.consumers.kafka.RecordOffsetObject; import org.apache.avro.file.DataFileReader; @@ -10,7 +11,10 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.apache.hadoop.hdfs.MiniDFSCluster; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeAll; @@ -23,22 +27,48 @@ public class HdfsTest { + private static MiniDFSCluster hdfsCluster; + private static File baseDir; + private static Config config; + // Generate AVRO-files for testing the HDFS writes. @BeforeAll public static void generateTestData() throws IOException, InterruptedException { - Config config = new Config(); + config = null; + try { + config = new Config(); + } catch (IOException e){ + System.out.println("Can't load config: " + e); + System.exit(1); + } catch (IllegalArgumentException e) { + System.out.println("Got invalid config: " + e); + System.exit(1); + } + startMiniCluster(); config.setMaximumFileSize(3000); // 10 loops (140 records) are in use at the moment, and that is sized at 36,102 bits. KafkaController kafkaController = new KafkaController(config); kafkaController.run(); } + public static void startMiniCluster() throws IOException { + // TODO: Create a HDFS miniCluster + baseDir = Files.createTempDirectory("test_hdfs").toFile().getAbsoluteFile(); + Configuration conf = new Configuration(); + conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, baseDir.getAbsolutePath()); + MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf); + hdfsCluster = builder.build(); + String hdfsURI = "hdfs://localhost:"+ hdfsCluster.getNameNodePort() + "/"; + System.out.println("hdfsURI: " + hdfsURI); + config.setHdfsuri(hdfsURI); + DistributedFileSystem fileSystem = hdfsCluster.getFileSystem(); + } + // Delete the generated AVRO-files. @AfterAll public static void deleteTestData() throws IOException { - Config config = new Config(); Path queueDirectory = new Path(config.getQueueDirectory()); // Paths.get(config.getQueueDirectory()); for (int j = 0; j <= 9; j++) { - for (int i = 1; i <= 5; i++) { + for (int i = 1; i <= 2; i++) { File syslogFile = new File( queueDirectory.toUri() + File.separator @@ -54,19 +84,28 @@ public static void deleteTestData() throws IOException { } } } + closeMiniCluster(); + } + + public static void closeMiniCluster() { + // TODO: teardown HDFS miniCluster + hdfsCluster.shutdown(); + FileUtil.fullyDelete(baseDir); + } + + // @Test + public void miniClusterDebugging() throws InterruptedException, IOException { + startMiniCluster(); + closeMiniCluster(); } @Test public void hdfsWriteTest() { - Config config = null; + try { - config = new Config(); - } catch (IOException e){ - System.out.println("Can't load config: " + e); - System.exit(1); - } catch (IllegalArgumentException e) { - System.out.println("Got invalid config: " + e); - System.exit(1); + startMiniCluster(); + } catch (IOException e) { + throw new RuntimeException(e); } Path queueDirectory = new Path(config.getQueueDirectory()); @@ -95,37 +134,31 @@ public void hdfsWriteTest() { assert lastRecord != null; RecordOffsetObject lastObject = new RecordOffsetObject("testConsumerTopic", Integer.parseInt(lastRecord.getPartition().toString()), lastRecord.getOffset(), null); // Fetch input parameters from the lastRecord SyslogRecord-object. System.out.println("Last record in the " + syslogFile.getName() + " file:" + "\ntopic: " + lastObject.topic + "\npartition: " + lastObject.partition + "\noffset: " + lastObject.offset + "\n"); - /*try (HDFSWriter writer = new HDFSWriter(config, lastObject)) { + // TODO: TEST THE WRITES! THE URL FOR HDFS MUST BE UPDATED TO THE URL THAT IS GENERATED IN startMiniCluster(). + try (HDFSWriter writer = new HDFSWriter(config, lastObject)) { + System.out.print("Testing stuff here."); writer.commit(syslogFile); // commits the final AVRO-file to HDFS. } catch (IOException e) { throw new RuntimeException(e); - }*/ + } // Check that the file was stored to HDFS properly. try { - wait(1000); + Thread.sleep(1000); hdfsReadCheck("testConsumerTopic", Integer.parseInt(lastRecord.getPartition().toString()), lastRecord.getOffset()); - } catch (InterruptedException | IOException e) { + } catch (IOException e) { + throw new RuntimeException(e); + } catch (InterruptedException e) { throw new RuntimeException(e); } } } + closeMiniCluster(); } public void hdfsReadCheck(String testConsumerTopic, int partition, long offset) throws IOException { - Config config = null; - try { - config = new Config(); - } catch (IOException e){ - System.out.println("Can't load config: " + e); - System.exit(1); - } catch (IllegalArgumentException e) { - System.out.println("Got invalid config: " + e); - System.exit(1); - } - // Check that the files were properly written to HDFS with a read test. String hdfsuri = config.getHdfsuri(); @@ -165,41 +198,56 @@ public void hdfsReadCheck(String testConsumerTopic, int partition, long offset) //The data is in AVRO-format, so it can't be read as a string. DataFileStream reader = new DataFileStream<>(inputStream, new SpecificDatumReader<>(SyslogRecord.class)); SyslogRecord record = null; + int looper = 0; while (reader.hasNext()) { reader.next(); record = reader.next(record); System.out.println(record); // Assert records here like it is done in KafkaConsumerTest.avroReader(). - if (offset <= 0) { + // FIXME: The offset looper is broken. Check KafkaConsumerTest.avroReader and compare it with this implementation. + /*if (looper <= 0) { Assertions.assertEquals("{\"timestamp\": 1650872090804000, \"message\": \"[WARN] 2022-04-25 07:34:50,804 com.teragrep.jla_02.Log4j Log - Log4j warn says hi!\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partition + "\", \"offset\": 0, \"origin\": \"jla-02.default\"}", record.toString()); - } else if (offset == 1) { + looper++; + } else if (looper == 1) { Assertions.assertEquals("{\"timestamp\": 1650872090806000, \"message\": \"[ERROR] 2022-04-25 07:34:50,806 com.teragrep.jla_02.Log4j Log - Log4j error says hi!\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partition + "\", \"offset\": 1, \"origin\": \"jla-02.default\"}", record.toString()); - } else if (offset == 2) { + looper++; + } else if (looper == 2) { Assertions.assertEquals("{\"timestamp\": 1650872090822000, \"message\": \"470647 [Thread-3] INFO com.teragrep.jla_02.Logback Daily - Logback-daily says hi.\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partition + "\", \"offset\": 2, \"origin\": \"jla-02\"}", record.toString()); - } else if (offset == 3) { + looper++; + } else if (looper == 3) { Assertions.assertEquals("{\"timestamp\": 1650872090822000, \"message\": \"470646 [Thread-3] INFO com.teragrep.jla_02.Logback Audit - Logback-audit says hi.\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partition + "\", \"offset\": 3, \"origin\": \"jla-02\"}", record.toString()); - } else if (offset == 4) { + looper++; + } else if (looper == 4) { Assertions.assertEquals("{\"timestamp\": 1650872090822000, \"message\": \"470647 [Thread-3] INFO com.teragrep.jla_02.Logback Metric - Logback-metric says hi.\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partition + "\", \"offset\": 4, \"origin\": \"jla-02\"}", record.toString()); - } else if (offset == 5) { + looper++; + } else if (looper == 5) { Assertions.assertEquals("{\"timestamp\": 1650872092238000, \"message\": \"25.04.2022 07:34:52.238 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info audit says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partition + "\", \"offset\": 5, \"origin\": \"jla-02.default\"}", record.toString()); - } else if (offset == 6) { + looper++; + } else if (looper == 6) { Assertions.assertEquals("{\"timestamp\": 1650872092239000, \"message\": \"25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info daily says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partition + "\", \"offset\": 6, \"origin\": \"jla-02.default\"}", record.toString()); - } else if (offset == 7) { + looper++; + } else if (looper == 7) { Assertions.assertEquals("{\"timestamp\": 1650872092239000, \"message\": \"25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info metric says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partition + "\", \"offset\": 7, \"origin\": \"jla-02.default\"}", record.toString()); - } else if (offset == 8) { + looper++; + } else if (looper == 8) { Assertions.assertEquals("{\"timestamp\": 1650872092240000, \"message\": \"25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn audit says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partition + "\", \"offset\": 8, \"origin\": \"jla-02.default\"}", record.toString()); - } else if (offset == 9) { + looper++; + } else if (looper == 9) { Assertions.assertEquals("{\"timestamp\": 1650872092240000, \"message\": \"25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn daily says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partition + "\", \"offset\": 9, \"origin\": \"jla-02.default\"}", record.toString()); - } else if (offset == 10) { + looper++; + } else if (looper == 10) { Assertions.assertEquals("{\"timestamp\": 1650872092241000, \"message\": \"25.04.2022 07:34:52.241 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn metric says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partition + "\", \"offset\": 10, \"origin\": \"jla-02.default\"}", record.toString()); - } else if (offset == 11) { + looper++; + } else if (looper == 11) { Assertions.assertEquals("{\"timestamp\": 1650872092241000, \"message\": \"25.04.2022 07:34:52.241 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error audit says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partition + "\", \"offset\": 11, \"origin\": \"jla-02.default\"}", record.toString()); - } else if (offset == 12) { + looper++; + } else if (looper == 12) { Assertions.assertEquals("{\"timestamp\": 1650872092242000, \"message\": \"25.04.2022 07:34:52.242 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error daily says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partition + "\", \"offset\": 12, \"origin\": \"jla-02.default\"}", record.toString()); + looper++; } else { Assertions.assertEquals("{\"timestamp\": 1650872092243000, \"message\": \"25.04.2022 07:34:52.243 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error metric says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partition + "\", \"offset\": 13, \"origin\": \"jla-02.default\"}", record.toString()); - } - + looper = 0; + }*/ } // logger.info(out); inputStream.close(); diff --git a/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java b/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java index 5d13c7d2..57f28121 100644 --- a/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java +++ b/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java @@ -65,7 +65,7 @@ public void kafkaAndAvroFullTest() throws InterruptedException { } catch (IOException e) { throw new RuntimeException(e); } - cleanup(config, 1, 5); + cleanup(config, 1, 2); } // Reads the data from a list of avro files From 6c9061f1601983fabece1085debb0af4671bf71d Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Tue, 6 Feb 2024 10:36:06 +0200 Subject: [PATCH 038/146] Fixed HDFS tests. --- .../java/com/teragrep/cfe_39/HdfsTest.java | 24 +++++++++++-------- .../teragrep/cfe_39/KafkaConsumerTest.java | 4 ++-- 2 files changed, 16 insertions(+), 12 deletions(-) diff --git a/src/test/java/com/teragrep/cfe_39/HdfsTest.java b/src/test/java/com/teragrep/cfe_39/HdfsTest.java index 08402636..50c2a0da 100644 --- a/src/test/java/com/teragrep/cfe_39/HdfsTest.java +++ b/src/test/java/com/teragrep/cfe_39/HdfsTest.java @@ -51,7 +51,7 @@ public static void generateTestData() throws IOException, InterruptedException { } public static void startMiniCluster() throws IOException { - // TODO: Create a HDFS miniCluster + // Create a HDFS miniCluster baseDir = Files.createTempDirectory("test_hdfs").toFile().getAbsoluteFile(); Configuration conf = new Configuration(); conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, baseDir.getAbsolutePath()); @@ -88,7 +88,7 @@ public static void deleteTestData() throws IOException { } public static void closeMiniCluster() { - // TODO: teardown HDFS miniCluster + // Teardown HDFS miniCluster hdfsCluster.shutdown(); FileUtil.fullyDelete(baseDir); } @@ -133,10 +133,8 @@ public void hdfsWriteTest() { assert lastRecord != null; RecordOffsetObject lastObject = new RecordOffsetObject("testConsumerTopic", Integer.parseInt(lastRecord.getPartition().toString()), lastRecord.getOffset(), null); // Fetch input parameters from the lastRecord SyslogRecord-object. - System.out.println("Last record in the " + syslogFile.getName() + " file:" + "\ntopic: " + lastObject.topic + "\npartition: " + lastObject.partition + "\noffset: " + lastObject.offset + "\n"); - // TODO: TEST THE WRITES! THE URL FOR HDFS MUST BE UPDATED TO THE URL THAT IS GENERATED IN startMiniCluster(). + System.out.println("\n"+"Last record in the " + syslogFile.getName() + " file:" + "\ntopic: " + lastObject.topic + "\npartition: " + lastObject.partition + "\noffset: " + lastObject.offset); try (HDFSWriter writer = new HDFSWriter(config, lastObject)) { - System.out.print("Testing stuff here."); writer.commit(syslogFile); // commits the final AVRO-file to HDFS. } catch (IOException e) { throw new RuntimeException(e); @@ -198,14 +196,20 @@ public void hdfsReadCheck(String testConsumerTopic, int partition, long offset) //The data is in AVRO-format, so it can't be read as a string. DataFileStream reader = new DataFileStream<>(inputStream, new SpecificDatumReader<>(SyslogRecord.class)); SyslogRecord record = null; - int looper = 0; + int looper; + if (offset == 8) { + looper = 0; + } else if (offset == 13) { + looper = 9; + }else { + looper = 0; + Assertions.fail("The offset of the last record is not 8 or 13, which means a failed test."); + } while (reader.hasNext()) { - reader.next(); record = reader.next(record); System.out.println(record); // Assert records here like it is done in KafkaConsumerTest.avroReader(). - // FIXME: The offset looper is broken. Check KafkaConsumerTest.avroReader and compare it with this implementation. - /*if (looper <= 0) { + if (looper <= 0) { Assertions.assertEquals("{\"timestamp\": 1650872090804000, \"message\": \"[WARN] 2022-04-25 07:34:50,804 com.teragrep.jla_02.Log4j Log - Log4j warn says hi!\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partition + "\", \"offset\": 0, \"origin\": \"jla-02.default\"}", record.toString()); looper++; } else if (looper == 1) { @@ -247,7 +251,7 @@ record = reader.next(record); } else { Assertions.assertEquals("{\"timestamp\": 1650872092243000, \"message\": \"25.04.2022 07:34:52.243 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error metric says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partition + "\", \"offset\": 13, \"origin\": \"jla-02.default\"}", record.toString()); looper = 0; - }*/ + } } // logger.info(out); inputStream.close(); diff --git a/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java b/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java index 57f28121..a10769ea 100644 --- a/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java +++ b/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java @@ -25,7 +25,7 @@ public class KafkaConsumerTest { // Make sure application.properties has consumer.useMockKafkaConsumer=true enabled for Kafka testing. - @Test + // @Test public void configTest() { // Configuration tests done, configurations working correctly with the right .jaas and .properties files. try { @@ -44,7 +44,7 @@ public void configTest() { } - @Test + // @Test public void kafkaAndAvroFullTest() throws InterruptedException { Config config = null; try { From a74131e08ebc6be6435188bde057e20976410d6c Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Tue, 6 Feb 2024 15:38:43 +0200 Subject: [PATCH 039/146] Created integration test for Kafka consumer groups, record processing and HDFS write. Fixed bugs in HDFS write logic. --- .../consumers/kafka/DatabaseOutput.java | 29 +-- .../cfe_39/consumers/kafka/HDFSWriter.java | 2 + .../com/teragrep/cfe_39/CombinedFullTest.java | 189 ++++++++++++++++++ .../java/com/teragrep/cfe_39/HdfsTest.java | 6 +- 4 files changed, 209 insertions(+), 17 deletions(-) create mode 100644 src/test/java/com/teragrep/cfe_39/CombinedFullTest.java diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java index 000d7c6b..0c29f7ce 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java @@ -106,7 +106,7 @@ public class DatabaseOutput implements Consumer> { this.approximatedSize = 0; } - boolean checkSizeTooLarge(long fileSize, RecordOffsetObject lastObject) { + boolean checkSizeTooLarge(long fileSize, RecordOffsetObject recordOffsetObject) { try { // If the syslogAvroWriter is already initialized, check the filesize so it doesn't go above maximumFileSize. if (fileSize > maximumFileSize) { @@ -115,14 +115,14 @@ boolean checkSizeTooLarge(long fileSize, RecordOffsetObject lastObject) { // This part closes the writing of now "complete" AVRO-file and stores the file to HDFS. syslogAvroWriter.close(); // FIXME: UNCOMMENT WHEN DONE TESTING THE CONSUMER! This commits the AVRO-file to HDFS. - /*try (HDFSWriter writer = new HDFSWriter(config, lastObject)) { + // FIXME: lastObject is faulty, it gives the wrong offset value to the HDFSWriter. + try (HDFSWriter writer = new HDFSWriter(config, recordOffsetObject)) { writer.commit(syslogFile); // commits the final AVRO-file to HDFS. - }*/ + } // This part defines a new empty file to which the new AVRO-serialized records are stored until it again hits the 64M size limit. - writableQueue.setQueueNamePrefix(lastObject.topic+lastObject.partition); - File syslogFile = - writableQueue.getNextWritableFile(); + writableQueue.setQueueNamePrefix(recordOffsetObject.topic+recordOffsetObject.partition); + syslogFile = writableQueue.getNextWritableFile(); syslogAvroWriter = new SyslogAvroWriter(syslogFile); approximatedSize = syslogAvroWriter.getFileSize(); // resets the size approximation. return true; @@ -189,19 +189,19 @@ public void accept(List recordOffsetObjectList) { assert lastObject != null; if (lastObject.topic.equals(recordOffsetObject.topic) & lastObject.partition.equals(recordOffsetObject.partition)) { // Records left to consume in the current partition. - checkSizeTooLarge(syslogAvroWriter.getFileSize(), recordOffsetObject); + checkSizeTooLarge(syslogAvroWriter.getFileSize(), lastObject); } else { // Previous partition was fully consumed. Commit file to HDFS and create a new AVRO-file. syslogAvroWriter.close(); // FIXME: UNCOMMENT WHEN DONE TESTING THE CONSUMER! This commits the AVRO-file to HDFS. - /*try (HDFSWriter writer = new HDFSWriter(config, lastObject)) { + // FIXME: lastObject is faulty, it gives the wrong offset value to the HDFSWriter. + try (HDFSWriter writer = new HDFSWriter(config, lastObject)) { writer.commit(syslogFile); - }*/ + } // This part defines a new empty file to which the new AVRO-serialized records are stored until it again hits the 64M size limit. writableQueue.setQueueNamePrefix(recordOffsetObject.topic+recordOffsetObject.partition); - File syslogFile = - writableQueue.getNextWritableFile(); + syslogFile = writableQueue.getNextWritableFile(); syslogAvroWriter = new SyslogAvroWriter(syslogFile); approximatedSize = syslogAvroWriter.getFileSize(); // resets the size approximation. } @@ -261,7 +261,7 @@ public void accept(List recordOffsetObjectList) { long capacity = syslogRecord.toByteBuffer().capacity(); // Check if there is still room in syslogAvroWriter for another syslogRecord. Commit syslogAvroWriter to HDFS if no room left, emptying it out in the process. // checkSizeTooLarge(approximatedSize + capacity, lastObject); // FIXME: approximatedSize is not working properly without the use of flush() after append. File sizes are all over the place. - checkSizeTooLarge(syslogAvroWriter.getFileSize() + capacity, recordOffsetObject); + checkSizeTooLarge(syslogAvroWriter.getFileSize() + capacity, lastObject); // Add syslogRecord to syslogAvroWriter which has room for new syslogRecord. syslogAvroWriter.write(syslogRecord); approximatedSize += capacity; @@ -278,9 +278,10 @@ public void accept(List recordOffsetObjectList) { if (syslogAvroWriter != null) { syslogAvroWriter.close(); // FIXME: UNCOMMENT WHEN DONE TESTING THE CONSUMER! This commits the AVRO-file to HDFS. - /*try (HDFSWriter writer = new HDFSWriter(config, lastObject)) { + // FIXME: lastObject is faulty, it gives the wrong offset value to the HDFSWriter. + try (HDFSWriter writer = new HDFSWriter(config, lastObject)) { writer.commit(syslogFile); // commits the final AVRO-file to HDFS. - }*/ + } } } catch (IOException e) { throw new RuntimeException(e); diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java index 677cb811..ef0d289c 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java @@ -162,6 +162,8 @@ public void commit(File syslogFile) { outputStream.close(); // logger.info("End Write file into hdfs"); boolean delete = syslogFile.delete(); // deletes the avro-file from the local disk now that it has been committed to HDFS. + // FIXME: delete is breaking something in integration test. + System.out.println("\n" + "File committed to HDFS, file writepath should be: " + hdfswritepath.toString() + "\n"); } catch (IOException e) { throw new RuntimeException(e); diff --git a/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java b/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java new file mode 100644 index 00000000..43a2de20 --- /dev/null +++ b/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java @@ -0,0 +1,189 @@ +package com.teragrep.cfe_39; + +import com.teragrep.cfe_39.avro.SyslogRecord; +import com.teragrep.cfe_39.consumers.kafka.HDFSWriter; +import com.teragrep.cfe_39.consumers.kafka.KafkaController; +import com.teragrep.cfe_39.consumers.kafka.RecordOffsetObject; +import org.apache.avro.file.DataFileReader; +import org.apache.avro.file.DataFileStream; +import org.apache.avro.io.DatumReader; +import org.apache.avro.specific.SpecificDatumReader; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FileUtil; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +import java.io.File; +import java.io.IOException; +import java.net.URI; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.List; + +public class CombinedFullTest { + + private static MiniDFSCluster hdfsCluster; + private static File baseDir; + private static Config config; + + + // Start minicluster and initialize config. + @BeforeAll + public static void startMiniCluster() throws IOException, InterruptedException { + config = null; + try { + config = new Config(); + } catch (IOException e){ + System.out.println("Can't load config: " + e); + System.exit(1); + } catch (IllegalArgumentException e) { + System.out.println("Got invalid config: " + e); + System.exit(1); + } + // Create a HDFS miniCluster + baseDir = Files.createTempDirectory("test_hdfs").toFile().getAbsoluteFile(); + Configuration conf = new Configuration(); + conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, baseDir.getAbsolutePath()); + MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf); + hdfsCluster = builder.build(); + String hdfsURI = "hdfs://localhost:"+ hdfsCluster.getNameNodePort() + "/"; + System.out.println("hdfsURI: " + hdfsURI); + config.setHdfsuri(hdfsURI); + DistributedFileSystem fileSystem = hdfsCluster.getFileSystem(); + } + + // Teardown the minicluster + @AfterAll + public static void teardownMiniCluster() { + hdfsCluster.shutdown(); + FileUtil.fullyDelete(baseDir); + } + + @Test + public void kafkaAndAvroFullTest() throws InterruptedException { + config.setMaximumFileSize(3000); // 10 loops (140 records) are in use at the moment, and that is sized at 36,102 bits. + KafkaController kafkaController = new KafkaController(config); + Thread.sleep(10000); + kafkaController.run(); + // The avro files should be committed to HDFS now. Check the committed files for any errors. + // There should be 20 files, 10 partitions with each having 2 files assigned to them. + try { + hdfsReadCheck(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + public void hdfsReadCheck() throws IOException { + // Check that the files were properly written to HDFS with a read test. + String hdfsuri = config.getHdfsuri(); + + String path = config.getHdfsPath()+"/"+"testConsumerTopic"; + // String fileName = "testConsumerTopic1.1"; + // ====== Init HDFS File System Object + Configuration conf = new Configuration(); + // Set FileSystem URI + conf.set("fs.defaultFS", hdfsuri); + // Because of Maven + conf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); + conf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); + // Set HADOOP user + System.setProperty("HADOOP_USER_NAME", "hdfs"); + System.setProperty("hadoop.home.dir", "/"); + //Get the filesystem - HDFS + FileSystem fs = FileSystem.get(URI.create(hdfsuri), conf); + + //==== Create folder if not exists + Path workingDir=fs.getWorkingDirectory(); + Path newFolderPath= new Path(path); + if(!fs.exists(newFolderPath)) { + // Create new Directory + fs.mkdirs(newFolderPath); + // logger.info("Path "+path+" created."); + } + + // This is the HDFS write path for the files: + // Path hdfswritepath = new Path(newFolderPath + "/" + fileName); where newFolderPath is config.getHdfsPath() + "/" + lastObject.topic; and filename is lastObject.partition+"."+lastObject.offset; + + + List filenameList = new ArrayList<>(); + for (int i = 0; i <= 9; i++) { + filenameList.add(i + "." + 8); + filenameList.add(i + "." + 13); + } + int looper = 0; + int partitionCounter = 0; + for (String fileName : filenameList) { + //==== Read files + // logger.info("Read file into hdfs"); + //Create a path + Path hdfsreadpath = new Path(newFolderPath + "/" + fileName); // The path should be the same that was used in writing the file to HDFS. + //Init input stream + FSDataInputStream inputStream = fs.open(hdfsreadpath); + //The data is in AVRO-format, so it can't be read as a string. + DataFileStream reader = new DataFileStream<>(inputStream, new SpecificDatumReader<>(SyslogRecord.class)); + SyslogRecord record = null; + System.out.println("\nReading records from file " + hdfsreadpath.toString() + ":"); + while (reader.hasNext()) { + record = reader.next(record); + System.out.println(record); + // Assert records here like it is done in KafkaConsumerTest.avroReader(). + if (looper <= 0) { + Assertions.assertEquals("{\"timestamp\": 1650872090804000, \"message\": \"[WARN] 2022-04-25 07:34:50,804 com.teragrep.jla_02.Log4j Log - Log4j warn says hi!\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 0, \"origin\": \"jla-02.default\"}", record.toString()); + looper++; + } else if (looper == 1) { + Assertions.assertEquals("{\"timestamp\": 1650872090806000, \"message\": \"[ERROR] 2022-04-25 07:34:50,806 com.teragrep.jla_02.Log4j Log - Log4j error says hi!\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 1, \"origin\": \"jla-02.default\"}", record.toString()); + looper++; + } else if (looper == 2) { + Assertions.assertEquals("{\"timestamp\": 1650872090822000, \"message\": \"470647 [Thread-3] INFO com.teragrep.jla_02.Logback Daily - Logback-daily says hi.\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 2, \"origin\": \"jla-02\"}", record.toString()); + looper++; + } else if (looper == 3) { + Assertions.assertEquals("{\"timestamp\": 1650872090822000, \"message\": \"470646 [Thread-3] INFO com.teragrep.jla_02.Logback Audit - Logback-audit says hi.\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 3, \"origin\": \"jla-02\"}", record.toString()); + looper++; + } else if (looper == 4) { + Assertions.assertEquals("{\"timestamp\": 1650872090822000, \"message\": \"470647 [Thread-3] INFO com.teragrep.jla_02.Logback Metric - Logback-metric says hi.\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 4, \"origin\": \"jla-02\"}", record.toString()); + looper++; + } else if (looper == 5) { + Assertions.assertEquals("{\"timestamp\": 1650872092238000, \"message\": \"25.04.2022 07:34:52.238 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info audit says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 5, \"origin\": \"jla-02.default\"}", record.toString()); + looper++; + } else if (looper == 6) { + Assertions.assertEquals("{\"timestamp\": 1650872092239000, \"message\": \"25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info daily says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 6, \"origin\": \"jla-02.default\"}", record.toString()); + looper++; + } else if (looper == 7) { + Assertions.assertEquals("{\"timestamp\": 1650872092239000, \"message\": \"25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info metric says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 7, \"origin\": \"jla-02.default\"}", record.toString()); + looper++; + } else if (looper == 8) { + Assertions.assertEquals("{\"timestamp\": 1650872092240000, \"message\": \"25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn audit says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 8, \"origin\": \"jla-02.default\"}", record.toString()); + looper++; + } else if (looper == 9) { + Assertions.assertEquals("{\"timestamp\": 1650872092240000, \"message\": \"25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn daily says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 9, \"origin\": \"jla-02.default\"}", record.toString()); + looper++; + } else if (looper == 10) { + Assertions.assertEquals("{\"timestamp\": 1650872092241000, \"message\": \"25.04.2022 07:34:52.241 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn metric says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 10, \"origin\": \"jla-02.default\"}", record.toString()); + looper++; + } else if (looper == 11) { + Assertions.assertEquals("{\"timestamp\": 1650872092241000, \"message\": \"25.04.2022 07:34:52.241 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error audit says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 11, \"origin\": \"jla-02.default\"}", record.toString()); + looper++; + } else if (looper == 12) { + Assertions.assertEquals("{\"timestamp\": 1650872092242000, \"message\": \"25.04.2022 07:34:52.242 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error daily says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 12, \"origin\": \"jla-02.default\"}", record.toString()); + looper++; + } else { + Assertions.assertEquals("{\"timestamp\": 1650872092243000, \"message\": \"25.04.2022 07:34:52.243 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error metric says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 13, \"origin\": \"jla-02.default\"}", record.toString()); + looper = 0; + partitionCounter++; + } + } + inputStream.close(); + // fs.close(); + } + fs.close(); + // logger.info(out); + } +} diff --git a/src/test/java/com/teragrep/cfe_39/HdfsTest.java b/src/test/java/com/teragrep/cfe_39/HdfsTest.java index 50c2a0da..4a7508a8 100644 --- a/src/test/java/com/teragrep/cfe_39/HdfsTest.java +++ b/src/test/java/com/teragrep/cfe_39/HdfsTest.java @@ -32,7 +32,7 @@ public class HdfsTest { private static Config config; // Generate AVRO-files for testing the HDFS writes. - @BeforeAll + // @BeforeAll public static void generateTestData() throws IOException, InterruptedException { config = null; try { @@ -64,7 +64,7 @@ public static void startMiniCluster() throws IOException { } // Delete the generated AVRO-files. - @AfterAll + // @AfterAll public static void deleteTestData() throws IOException { Path queueDirectory = new Path(config.getQueueDirectory()); // Paths.get(config.getQueueDirectory()); for (int j = 0; j <= 9; j++) { @@ -99,7 +99,7 @@ public void miniClusterDebugging() throws InterruptedException, IOException { closeMiniCluster(); } - @Test + // @Test public void hdfsWriteTest() { try { From 29f82e99a151efb283abb89865cdffbe9f93bf74 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Wed, 7 Feb 2024 16:27:54 +0200 Subject: [PATCH 040/146] Cleaning up comments. Adding missing loggers to files. --- .../consumers/kafka/DatabaseOutput.java | 20 +++---------- .../cfe_39/consumers/kafka/HDFSWriter.java | 18 +++++++----- .../consumers/kafka/KafkaController.java | 2 -- .../cfe_39/consumers/kafka/KafkaReader.java | 3 -- .../com/teragrep/cfe_39/CombinedFullTest.java | 29 +++++++++---------- .../java/com/teragrep/cfe_39/HdfsTest.java | 16 ++++++---- .../teragrep/cfe_39/KafkaConsumerTest.java | 17 ++++++----- 7 files changed, 47 insertions(+), 58 deletions(-) diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java index 0c29f7ce..871c9a6b 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java @@ -35,8 +35,6 @@ import java.nio.ByteBuffer; -// TODO: CLEAN COMMENTS, AND ADD COMMENTS WHERE NEEDED. - // The kafka stream should first be deserialized using rlo_06 and then serialized again using avro and stored in HDFS. // The target where the record is stored in HDFS is based on the topic, partition and offset. ie. topic_name/0.123456 where offset is 123456 @@ -106,6 +104,7 @@ public class DatabaseOutput implements Consumer> { this.approximatedSize = 0; } + // Checks that the filesize stays under the defined maximum file size. boolean checkSizeTooLarge(long fileSize, RecordOffsetObject recordOffsetObject) { try { // If the syslogAvroWriter is already initialized, check the filesize so it doesn't go above maximumFileSize. @@ -114,8 +113,6 @@ boolean checkSizeTooLarge(long fileSize, RecordOffsetObject recordOffsetObject) // This part closes the writing of now "complete" AVRO-file and stores the file to HDFS. syslogAvroWriter.close(); - // FIXME: UNCOMMENT WHEN DONE TESTING THE CONSUMER! This commits the AVRO-file to HDFS. - // FIXME: lastObject is faulty, it gives the wrong offset value to the HDFSWriter. try (HDFSWriter writer = new HDFSWriter(config, recordOffsetObject)) { writer.commit(syslogFile); // commits the final AVRO-file to HDFS. } @@ -145,9 +142,9 @@ private long rfc3339ToEpoch(ZonedDateTime zonedDateTime) { // Input parameter is a list of RecordOffsetObjects. Each object contains a record and its metadata (topic, partition and offset). // Each partition will get their set of exclusive AVRO-files in HDFS. - // The target where the record is stored in HDFS is based on the topic, partition and last offset. ie. topic_name/0.123456 where last written records offset is 123456. + // The target where the record is stored in HDFS is based on the topic, partition and last offset. ie. topic_name/0.123456 where last written record's offset is 123456. // AVRO-file with a path/name that starts with topic_name/0.X should only contain records from the 0th partition of topic named topic_name, topic_name/1.X should only contain records from 1st partition, etc. - // AVRO-files are created dynamically, thus it is not known which record (and its offset) is written to the file last before committing it to HDFS. The final name (specifically the offset) is decided only when the file is committed to HDFS. + // AVRO-files are created dynamically, thus it is not known which record (and its offset) is written to the file last before committing it to HDFS. The final name for the HDFS file is decided only when the file is committed to HDFS. @Override public void accept(List recordOffsetObjectList) { long thisTime = Instant.now().toEpochMilli(); @@ -160,9 +157,6 @@ public void accept(List recordOffsetObjectList) { // While it goes through the list, the contents of the objects are serialized into an AVRO-file. // When the file size is about to go above 64M, commit the file into HDFS using the latest topic/partition/offset values as the filename and start fresh with a new empty AVRO-file. // Serialize the object that was going to make the file go above 64M into the now empty AVRO-file and continue the loop. - // https://pagure.xnet.fi/com-teragrep/rlo_09/blob/avroness/f/src/main/java/com/teragrep/rlo_09/SyslogAvroWriter.java - // https://pagure.xnet.fi/com-teragrep/rlo_09/blob/avroness/f/src/main/java/com/teragrep/rlo_09/WriteCoordinator.java - // every recordOffsetObject.record on the recordOffsetObjectList basically represents a rlo_09 WriteCoordinator.accept(byte[] bytes) when the list is gone through in a loop. // TODO: If the prod-environment recordOffsetObjectList ordering is different from what it is in the test environment, add a function that reorders the list based on partition and offset (or better yet, make several AVRO-files that are being used at the same time rather than doing it one AVRO-file at a time as the offset ordering within partitions should always be correct in all scenarios). RecordOffsetObject lastObject = null; long start = Instant.now().toEpochMilli(); // Starts measuring performance here. Measures how long it takes to process the whole recordOffsetObjectList. @@ -174,9 +168,7 @@ public void accept(List recordOffsetObjectList) { writableQueue.setQueueNamePrefix(recordOffsetObject.topic+recordOffsetObject.partition); syslogFile = writableQueue.getNextWritableFile(); - // The avro serialization filename shouldn't really matter as long as the name is changed when stuff is stored to HDFS. - // And the content of the AVRO-serialized file that is going to be stored in HDFS is finalized only when the maximumFileSize has been reached. - // This means the HDFS filename is only finalized when the AVRO-serialized file is finalized, because every Kafka-record added to the file is going to change the offset that is going to be used for the filename. + // The HDFS filename is only finalized when the AVRO-serialized file is finalized, because every Kafka-record added to the file is going to change the offset that is going to be used for the filename. syslogAvroWriter = new SyslogAvroWriter(syslogFile); approximatedSize = syslogAvroWriter.getFileSize(); // resets the size approximation. lastObject = recordOffsetObject; @@ -193,8 +185,6 @@ public void accept(List recordOffsetObjectList) { } else { // Previous partition was fully consumed. Commit file to HDFS and create a new AVRO-file. syslogAvroWriter.close(); - // FIXME: UNCOMMENT WHEN DONE TESTING THE CONSUMER! This commits the AVRO-file to HDFS. - // FIXME: lastObject is faulty, it gives the wrong offset value to the HDFSWriter. try (HDFSWriter writer = new HDFSWriter(config, lastObject)) { writer.commit(syslogFile); } @@ -277,8 +267,6 @@ public void accept(List recordOffsetObjectList) { try { if (syslogAvroWriter != null) { syslogAvroWriter.close(); - // FIXME: UNCOMMENT WHEN DONE TESTING THE CONSUMER! This commits the AVRO-file to HDFS. - // FIXME: lastObject is faulty, it gives the wrong offset value to the HDFSWriter. try (HDFSWriter writer = new HDFSWriter(config, lastObject)) { writer.commit(syslogFile); // commits the final AVRO-file to HDFS. } diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java index ef0d289c..63c3dafd 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java @@ -22,6 +22,8 @@ import org.apache.hadoop.fs.*; import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.security.UserGroupInformation; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.File; import java.io.IOException; @@ -32,6 +34,7 @@ public class HDFSWriter implements AutoCloseable{ + private static final Logger LOGGER = LoggerFactory.getLogger(HDFSWriter.class); private final String fileName; private final String path; private final FileSystem fs; @@ -136,11 +139,11 @@ public void commit(File syslogFile) { if (!fs.exists(newFolderPath)) { // Create new Directory fs.mkdirs(newFolderPath); - // logger.info("Path "+path+" created."); + LOGGER.info("Path "+path+" created."); } //==== Write file - // logger.info("Begin Write file into hdfs"); + LOGGER.info("Begin Write file into hdfs"); //Create a path Path hdfswritepath = new Path(newFolderPath + "/" + fileName); // filename should be set according to the requirements: 0.12345 where 0 is Kafka partition and 12345 is Kafka offset. if (fs.exists(hdfswritepath)) { @@ -160,10 +163,9 @@ public void commit(File syslogFile) { outputStream.write(bytes); outputStream.close(); - // logger.info("End Write file into hdfs"); + LOGGER.info("End Write file into hdfs"); boolean delete = syslogFile.delete(); // deletes the avro-file from the local disk now that it has been committed to HDFS. - // FIXME: delete is breaking something in integration test. - System.out.println("\n" + "File committed to HDFS, file writepath should be: " + hdfswritepath.toString() + "\n"); + // System.out.println("\n" + "File committed to HDFS, file writepath should be: " + hdfswritepath.toString() + "\n"); } catch (IOException e) { throw new RuntimeException(e); @@ -178,11 +180,11 @@ public void commit(File syslogFile) { if (!fs.exists(newFolderPath)) { // Create new Directory fs.mkdirs(newFolderPath); - // logger.info("Path "+path+" created."); + LOGGER.info("Path "+path+" created."); } //==== Write file - // logger.info("Begin Write file into hdfs"); + LOGGER.info("Begin Write file into hdfs"); //Create a path Path hdfswritepath = new Path(newFolderPath + "/" + fileName); // filename should be set according to the requirements: 0.12345 where 0 is Kafka partition and 12345 is Kafka offset. //Init output stream @@ -199,7 +201,7 @@ public void commit(File syslogFile) { byte[] bytes = Files.readAllBytes(Paths.get(syslogFile.getPath())); outputStream.write(bytes); outputStream.close(); - // logger.info("End Write file into hdfs"); + LOGGER.info("End Write file into hdfs"); boolean delete = syslogFile.delete(); // deletes the avro-file from the local disk now that it has been committed to HDFS. } catch (IOException e) { diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java index dc1c435d..555f1cbd 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java @@ -63,9 +63,7 @@ public class KafkaController { // processes ONLY those messages that have not already been processed based on the offset values stored in HDFS filenames. private static final Logger LOGGER = LoggerFactory.getLogger(KafkaController.class); - private final Config config; - private final org.apache.kafka.clients.consumer.Consumer kafkaConsumer; private final List threads = new ArrayList<>(); private final Set activeTopics = new HashSet<>(); diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReader.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReader.java index d4e6674d..9f3d3394 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReader.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReader.java @@ -27,11 +27,8 @@ public class KafkaReader implements AutoCloseable { final Logger LOGGER = LoggerFactory.getLogger(KafkaReader.class); - private Iterator> kafkaRecordsIterator = Collections.emptyIterator(); - private final Consumer kafkaConsumer; - private final java.util.function.Consumer> callbackFunction; public KafkaReader( diff --git a/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java b/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java index 43a2de20..74e4cd93 100644 --- a/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java +++ b/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java @@ -1,12 +1,9 @@ package com.teragrep.cfe_39; import com.teragrep.cfe_39.avro.SyslogRecord; -import com.teragrep.cfe_39.consumers.kafka.HDFSWriter; +import com.teragrep.cfe_39.consumers.kafka.DatabaseOutput; import com.teragrep.cfe_39.consumers.kafka.KafkaController; -import com.teragrep.cfe_39.consumers.kafka.RecordOffsetObject; -import org.apache.avro.file.DataFileReader; import org.apache.avro.file.DataFileStream; -import org.apache.avro.io.DatumReader; import org.apache.avro.specific.SpecificDatumReader; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; @@ -19,6 +16,8 @@ import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.File; import java.io.IOException; @@ -29,6 +28,7 @@ public class CombinedFullTest { + private static final Logger LOGGER = LoggerFactory.getLogger(CombinedFullTest.class); private static MiniDFSCluster hdfsCluster; private static File baseDir; private static Config config; @@ -41,10 +41,10 @@ public static void startMiniCluster() throws IOException, InterruptedException { try { config = new Config(); } catch (IOException e){ - System.out.println("Can't load config: " + e); + LOGGER.error("Can't load config: " + e); System.exit(1); } catch (IllegalArgumentException e) { - System.out.println("Got invalid config: " + e); + LOGGER.error("Got invalid config: " + e); System.exit(1); } // Create a HDFS miniCluster @@ -54,7 +54,7 @@ public static void startMiniCluster() throws IOException, InterruptedException { MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf); hdfsCluster = builder.build(); String hdfsURI = "hdfs://localhost:"+ hdfsCluster.getNameNodePort() + "/"; - System.out.println("hdfsURI: " + hdfsURI); + // System.out.println("hdfsURI: " + hdfsURI); config.setHdfsuri(hdfsURI); DistributedFileSystem fileSystem = hdfsCluster.getFileSystem(); } @@ -68,7 +68,7 @@ public static void teardownMiniCluster() { @Test public void kafkaAndAvroFullTest() throws InterruptedException { - config.setMaximumFileSize(3000); // 10 loops (140 records) are in use at the moment, and that is sized at 36,102 bits. + config.setMaximumFileSize(3000); // 10 loops (140 records) are in use at the moment, and that is sized at 36,102 bytes. KafkaController kafkaController = new KafkaController(config); Thread.sleep(10000); kafkaController.run(); @@ -86,7 +86,6 @@ public void hdfsReadCheck() throws IOException { String hdfsuri = config.getHdfsuri(); String path = config.getHdfsPath()+"/"+"testConsumerTopic"; - // String fileName = "testConsumerTopic1.1"; // ====== Init HDFS File System Object Configuration conf = new Configuration(); // Set FileSystem URI @@ -106,13 +105,13 @@ public void hdfsReadCheck() throws IOException { if(!fs.exists(newFolderPath)) { // Create new Directory fs.mkdirs(newFolderPath); - // logger.info("Path "+path+" created."); + LOGGER.info("Path "+path+" created."); } // This is the HDFS write path for the files: // Path hdfswritepath = new Path(newFolderPath + "/" + fileName); where newFolderPath is config.getHdfsPath() + "/" + lastObject.topic; and filename is lastObject.partition+"."+lastObject.offset; - + // Create the list of files to read from HDFS. Test setup is created so each of the 0-9 partitions will have 2 files with offsets of 8 and 13. List filenameList = new ArrayList<>(); for (int i = 0; i <= 9; i++) { filenameList.add(i + "." + 8); @@ -122,7 +121,7 @@ public void hdfsReadCheck() throws IOException { int partitionCounter = 0; for (String fileName : filenameList) { //==== Read files - // logger.info("Read file into hdfs"); + LOGGER.info("Read file into hdfs"); //Create a path Path hdfsreadpath = new Path(newFolderPath + "/" + fileName); // The path should be the same that was used in writing the file to HDFS. //Init input stream @@ -130,10 +129,10 @@ public void hdfsReadCheck() throws IOException { //The data is in AVRO-format, so it can't be read as a string. DataFileStream reader = new DataFileStream<>(inputStream, new SpecificDatumReader<>(SyslogRecord.class)); SyslogRecord record = null; - System.out.println("\nReading records from file " + hdfsreadpath.toString() + ":"); + LOGGER.info("\nReading records from file " + hdfsreadpath.toString() + ":"); while (reader.hasNext()) { record = reader.next(record); - System.out.println(record); + LOGGER.info(record.toString()); // Assert records here like it is done in KafkaConsumerTest.avroReader(). if (looper <= 0) { Assertions.assertEquals("{\"timestamp\": 1650872090804000, \"message\": \"[WARN] 2022-04-25 07:34:50,804 com.teragrep.jla_02.Log4j Log - Log4j warn says hi!\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 0, \"origin\": \"jla-02.default\"}", record.toString()); @@ -181,9 +180,7 @@ record = reader.next(record); } } inputStream.close(); - // fs.close(); } fs.close(); - // logger.info(out); } } diff --git a/src/test/java/com/teragrep/cfe_39/HdfsTest.java b/src/test/java/com/teragrep/cfe_39/HdfsTest.java index 4a7508a8..8e8a539d 100644 --- a/src/test/java/com/teragrep/cfe_39/HdfsTest.java +++ b/src/test/java/com/teragrep/cfe_39/HdfsTest.java @@ -1,6 +1,7 @@ package com.teragrep.cfe_39; import com.teragrep.cfe_39.avro.SyslogRecord; +import com.teragrep.cfe_39.consumers.kafka.DatabaseOutput; import com.teragrep.cfe_39.consumers.kafka.HDFSWriter; import com.teragrep.cfe_39.consumers.kafka.KafkaController; import com.teragrep.cfe_39.consumers.kafka.RecordOffsetObject; @@ -19,6 +20,8 @@ import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.File; import java.io.IOException; @@ -26,6 +29,7 @@ import java.nio.file.Files; public class HdfsTest { + private static final Logger LOGGER = LoggerFactory.getLogger(HdfsTest.class); private static MiniDFSCluster hdfsCluster; private static File baseDir; @@ -38,14 +42,14 @@ public static void generateTestData() throws IOException, InterruptedException { try { config = new Config(); } catch (IOException e){ - System.out.println("Can't load config: " + e); + LOGGER.error("Can't load config: " + e); System.exit(1); } catch (IllegalArgumentException e) { - System.out.println("Got invalid config: " + e); + LOGGER.error("Got invalid config: " + e); System.exit(1); } startMiniCluster(); - config.setMaximumFileSize(3000); // 10 loops (140 records) are in use at the moment, and that is sized at 36,102 bits. + config.setMaximumFileSize(3000); // 10 loops (140 records) are in use at the moment, and that is sized at 36,102 bytes. KafkaController kafkaController = new KafkaController(config); kafkaController.run(); } @@ -58,7 +62,7 @@ public static void startMiniCluster() throws IOException { MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf); hdfsCluster = builder.build(); String hdfsURI = "hdfs://localhost:"+ hdfsCluster.getNameNodePort() + "/"; - System.out.println("hdfsURI: " + hdfsURI); + LOGGER.debug("hdfsURI: " + hdfsURI); config.setHdfsuri(hdfsURI); DistributedFileSystem fileSystem = hdfsCluster.getFileSystem(); } @@ -133,7 +137,7 @@ public void hdfsWriteTest() { assert lastRecord != null; RecordOffsetObject lastObject = new RecordOffsetObject("testConsumerTopic", Integer.parseInt(lastRecord.getPartition().toString()), lastRecord.getOffset(), null); // Fetch input parameters from the lastRecord SyslogRecord-object. - System.out.println("\n"+"Last record in the " + syslogFile.getName() + " file:" + "\ntopic: " + lastObject.topic + "\npartition: " + lastObject.partition + "\noffset: " + lastObject.offset); + LOGGER.debug("\n"+"Last record in the " + syslogFile.getName() + " file:" + "\ntopic: " + lastObject.topic + "\npartition: " + lastObject.partition + "\noffset: " + lastObject.offset); try (HDFSWriter writer = new HDFSWriter(config, lastObject)) { writer.commit(syslogFile); // commits the final AVRO-file to HDFS. } catch (IOException e) { @@ -207,7 +211,7 @@ public void hdfsReadCheck(String testConsumerTopic, int partition, long offset) } while (reader.hasNext()) { record = reader.next(record); - System.out.println(record); + LOGGER.debug(record.toString()); // Assert records here like it is done in KafkaConsumerTest.avroReader(). if (looper <= 0) { Assertions.assertEquals("{\"timestamp\": 1650872090804000, \"message\": \"[WARN] 2022-04-25 07:34:50,804 com.teragrep.jla_02.Log4j Log - Log4j warn says hi!\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partition + "\", \"offset\": 0, \"origin\": \"jla-02.default\"}", record.toString()); diff --git a/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java b/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java index a10769ea..3e5077aa 100644 --- a/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java +++ b/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java @@ -12,6 +12,8 @@ import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; import com.teragrep.cfe_39.avro.SyslogRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.nio.charset.StandardCharsets; import java.nio.file.Files; @@ -23,6 +25,7 @@ import java.util.*; public class KafkaConsumerTest { + private static final Logger LOGGER = LoggerFactory.getLogger(KafkaConsumerTest.class); // Make sure application.properties has consumer.useMockKafkaConsumer=true enabled for Kafka testing. // @Test @@ -37,7 +40,7 @@ public void configTest() { boolean useMockKafkaConsumer = Boolean.parseBoolean( readerKafkaProperties.getProperty("useMockKafkaConsumer", "false") ); - System.out.println("useMockKafkaConsumer: "+useMockKafkaConsumer); + LOGGER.debug("useMockKafkaConsumer: "+useMockKafkaConsumer); } catch (IOException e) { throw new RuntimeException(e); } @@ -50,13 +53,13 @@ public void kafkaAndAvroFullTest() throws InterruptedException { try { config = new Config(); } catch (IOException e){ - System.out.println("Can't load config: " + e); + LOGGER.error("Can't load config: " + e); System.exit(1); } catch (IllegalArgumentException e) { - System.out.println("Got invalid config: " + e); + LOGGER.error("Got invalid config: " + e); System.exit(1); } - config.setMaximumFileSize(3000); // 10 loops (140 records) are in use at the moment, and that is sized at 36,102 bits. + config.setMaximumFileSize(3000); // 10 loops (140 records) are in use at the moment, and that is sized at 36,102 bytes. KafkaController kafkaController = new KafkaController(config); kafkaController.run(); try { @@ -91,8 +94,8 @@ public int avroReader(int start, int end) throws IOException { SyslogRecord user = null; while (dataFileReader.hasNext()) { user = dataFileReader.next(user); - System.out.println(syslogFile.getPath()); - System.out.println(user); + LOGGER.debug(syslogFile.getPath()); + LOGGER.debug(user.toString()); counter++; // All the mock data is generated from a set of 14 records. if (looper <= 0) { @@ -143,7 +146,7 @@ public int avroReader(int start, int end) throws IOException { } } } - System.out.println("Total number of records: " + counter); + LOGGER.debug("Total number of records: " + counter); return counter; } From 8ca381f1b75f25c85f569b08153b538cbc01f24a Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Thu, 8 Feb 2024 12:41:09 +0200 Subject: [PATCH 041/146] Cleaning up loggers. Added missing exception handling to Kerberized HDFS write. --- .../cfe_39/consumers/kafka/HDFSWriter.java | 22 +++++++++++-------- .../consumers/kafka/KafkaController.java | 4 ++-- .../consumers/kafka/SyslogAvroWriter.java | 6 ++++- 3 files changed, 20 insertions(+), 12 deletions(-) diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java index 63c3dafd..f199b075 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java @@ -139,11 +139,11 @@ public void commit(File syslogFile) { if (!fs.exists(newFolderPath)) { // Create new Directory fs.mkdirs(newFolderPath); - LOGGER.info("Path "+path+" created."); + LOGGER.debug("Path "+path+" created."); } //==== Write file - LOGGER.info("Begin Write file into hdfs"); + LOGGER.debug("Begin Write file into hdfs"); //Create a path Path hdfswritepath = new Path(newFolderPath + "/" + fileName); // filename should be set according to the requirements: 0.12345 where 0 is Kafka partition and 12345 is Kafka offset. if (fs.exists(hdfswritepath)) { @@ -163,9 +163,9 @@ public void commit(File syslogFile) { outputStream.write(bytes); outputStream.close(); - LOGGER.info("End Write file into hdfs"); + LOGGER.debug("End Write file into hdfs"); boolean delete = syslogFile.delete(); // deletes the avro-file from the local disk now that it has been committed to HDFS. - // System.out.println("\n" + "File committed to HDFS, file writepath should be: " + hdfswritepath.toString() + "\n"); + LOGGER.debug("\n" + "File committed to HDFS, file writepath should be: " + hdfswritepath.toString() + "\n"); } catch (IOException e) { throw new RuntimeException(e); @@ -180,16 +180,19 @@ public void commit(File syslogFile) { if (!fs.exists(newFolderPath)) { // Create new Directory fs.mkdirs(newFolderPath); - LOGGER.info("Path "+path+" created."); + LOGGER.debug("Path "+path+" created."); } //==== Write file - LOGGER.info("Begin Write file into hdfs"); + LOGGER.debug("Begin Write file into hdfs"); //Create a path Path hdfswritepath = new Path(newFolderPath + "/" + fileName); // filename should be set according to the requirements: 0.12345 where 0 is Kafka partition and 12345 is Kafka offset. + if (fs.exists(hdfswritepath)) { + throw new RuntimeException("File " + fileName + " already exists"); + } + //Init output stream FSDataOutputStream outputStream = fs.create(hdfswritepath); - // Write the file contents of syslogFile to hdfswritepath in HDFS. // file to bytes[] @@ -197,12 +200,13 @@ public void commit(File syslogFile) { try (FileInputStream inputStream = new FileInputStream(syslogFile)) { inputStream.read(bytearray); }*/ - byte[] bytes = Files.readAllBytes(Paths.get(syslogFile.getPath())); outputStream.write(bytes); + outputStream.close(); - LOGGER.info("End Write file into hdfs"); + LOGGER.debug("End Write file into hdfs"); boolean delete = syslogFile.delete(); // deletes the avro-file from the local disk now that it has been committed to HDFS. + LOGGER.debug("\n" + "File committed to HDFS, file writepath should be: " + hdfswritepath.toString() + "\n"); } catch (IOException e) { throw new RuntimeException(e); diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java index 555f1cbd..f175c4cb 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java @@ -106,7 +106,7 @@ public void run() throws InterruptedException { // For testing purposes only. Stops the run when all the records are consumed from the mockConsumer during test. if (durationStatistics.getTotalRecords() > 0 & useMockKafkaConsumer) { - LOGGER.info("Processed all the test records. Closing."); + LOGGER.debug("Processed all the test records. Closing."); keepRunning = false; } @@ -184,7 +184,7 @@ private void topicScan(DurationStatistics durationStatistics, List // Activate all the found in-active topics, in other words create consumer groups for all of them using the createReader()-function. foundPartitions.forEach((k, v) -> { - LOGGER.info("Activating topic <"+k+">"); + LOGGER.debug("Activating topic <"+k+">"); try { createReader(k, v, topicCounters); activeTopics.add(k); diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java index 4eeb290c..4afa932b 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java @@ -24,12 +24,16 @@ import org.apache.avro.file.SyncableFileOutputStream; import org.apache.avro.io.DatumWriter; import org.apache.avro.specific.SpecificDatumWriter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.*; import java.nio.ByteBuffer; class SyslogAvroWriter implements AutoCloseable { + private static final Logger LOGGER = LoggerFactory.getLogger(SyslogAvroWriter.class); + private final DatumWriter datumWriter = new SpecificDatumWriter<>(SyslogRecord.class); @@ -44,7 +48,7 @@ class SyslogAvroWriter implements AutoCloseable { syncableFileOutputStream = new SyncableFileOutputStream(syslogFile); - // System.out.println("debugging syslogFile, path is: " + syslogFile.getPath()); + // LOGGER.debug("debugging syslogFile, path is: " + syslogFile.getPath()); syncableFileOutputStream.getChannel().tryLock(); if (syslogFile.length() == 0) { From e836a1fcbc1a409f414287123bef5b9451323529 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Thu, 8 Feb 2024 16:34:07 +0200 Subject: [PATCH 042/146] Testing FileSystem limitations on how pruning of the HDFS database can be achieved. --- .../com/teragrep/cfe_39/CombinedFullTest.java | 46 +++++++++++++++++-- 1 file changed, 41 insertions(+), 5 deletions(-) diff --git a/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java b/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java index 74e4cd93..e1ea0a33 100644 --- a/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java +++ b/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java @@ -6,10 +6,7 @@ import org.apache.avro.file.DataFileStream; import org.apache.avro.specific.SpecificDatumReader; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FSDataInputStream; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.FileUtil; -import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.*; import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.junit.jupiter.api.AfterAll; @@ -23,6 +20,7 @@ import java.io.IOException; import java.net.URI; import java.nio.file.Files; +import java.sql.Timestamp; import java.util.ArrayList; import java.util.List; @@ -74,11 +72,49 @@ public void kafkaAndAvroFullTest() throws InterruptedException { kafkaController.run(); // The avro files should be committed to HDFS now. Check the committed files for any errors. // There should be 20 files, 10 partitions with each having 2 files assigned to them. - try { + /*try { hdfsReadCheck(); } catch (IOException e) { throw new RuntimeException(e); + }*/ + } + + @Test + public void hdfsPruneTest() throws IOException { + // Check that the files were properly written to HDFS with a read test. + String hdfsuri = config.getHdfsuri(); + + String path = config.getHdfsPath()+"/"+"testConsumerTopic"; + // ====== Init HDFS File System Object + Configuration conf = new Configuration(); + // Set FileSystem URI + conf.set("fs.defaultFS", hdfsuri); + // Because of Maven + conf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); + conf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); + // Set HADOOP user + System.setProperty("HADOOP_USER_NAME", "hdfs"); + System.setProperty("hadoop.home.dir", "/"); + //Get the filesystem - HDFS + FileSystem fs = FileSystem.get(URI.create(hdfsuri), conf); + + //==== Create folder if not exists + Path workingDir=fs.getWorkingDirectory(); + Path newFolderPath= new Path(path); + if(!fs.exists(newFolderPath)) { + // Create new Directory + fs.mkdirs(newFolderPath); + LOGGER.info("Path "+path+" created."); } + + // TODO: Which timestamp should be used for pruning? The HDFS timestamp is easy to fetch and use but it is inaccurate especially for the first full cycle when the cfe_39 is first started. + // It would be much more heavy task to check the timestamps of the records that are stored in the file, rather than just checking the file timestamp. + FileStatus[] fileStatuses = fs.listStatus(new Path(newFolderPath + "/")); + for (FileStatus a : fileStatuses) { + Timestamp timestamp = new Timestamp(a.getModificationTime()); + LOGGER.info("Timestamp for file " + a.getPath() + " is: " + timestamp); + } + fs.close(); } public void hdfsReadCheck() throws IOException { From 4c274c52e034e48db8fd14f2d9be3afa03bce9e1 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Fri, 9 Feb 2024 13:02:36 +0200 Subject: [PATCH 043/146] Testing FileSystem limitations on how pruning of the HDFS database can be achieved. (2) --- .../java/com/teragrep/cfe_39/CombinedFullTest.java | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java b/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java index e1ea0a33..1c0eb10c 100644 --- a/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java +++ b/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java @@ -23,6 +23,7 @@ import java.sql.Timestamp; import java.util.ArrayList; import java.util.List; +import java.util.Objects; public class CombinedFullTest { @@ -109,10 +110,18 @@ public void hdfsPruneTest() throws IOException { // TODO: Which timestamp should be used for pruning? The HDFS timestamp is easy to fetch and use but it is inaccurate especially for the first full cycle when the cfe_39 is first started. // It would be much more heavy task to check the timestamps of the records that are stored in the file, rather than just checking the file timestamp. + + // TODO: To bypass the issue of modification timestamps being inaccurate there is the method of altering the modification timestamps to mirror the timestamp of the records during the initial commit of the file: + fs.setTimes(new Path(path+"/"+0.8), Long.parseUnsignedLong("1675930598000"), -1); // where mtime is modification time and atime is access time. -1 as input parameter leaves the original atime/mtime value as is. FileStatus[] fileStatuses = fs.listStatus(new Path(newFolderPath + "/")); for (FileStatus a : fileStatuses) { Timestamp timestamp = new Timestamp(a.getModificationTime()); - LOGGER.info("Timestamp for file " + a.getPath() + " is: " + timestamp); + String asfas = hdfsuri + "opt/teragrep/cfe_39/srv/testConsumerTopic/" + 0.8; + LOGGER.info(asfas); + if (Objects.equals(a.getPath().toString(), asfas)) { + Assertions.assertEquals(timestamp, new Timestamp(Long.parseUnsignedLong("1675930598000"))); + LOGGER.info("testConsumerTopic/0.8 passed assertion test, timestamp was properly set to 1675930598000"); + } } fs.close(); } From 045ff95d1353617580ffa2e03bafc78fa08f178f Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Mon, 12 Feb 2024 15:56:28 +0200 Subject: [PATCH 044/146] Setting up pruning using MapReduce. --- .../com/teragrep/cfe_39/CombinedFullTest.java | 21 +++++++++++++------ 1 file changed, 15 insertions(+), 6 deletions(-) diff --git a/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java b/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java index 1c0eb10c..79c8d700 100644 --- a/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java +++ b/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java @@ -73,11 +73,11 @@ public void kafkaAndAvroFullTest() throws InterruptedException { kafkaController.run(); // The avro files should be committed to HDFS now. Check the committed files for any errors. // There should be 20 files, 10 partitions with each having 2 files assigned to them. - /*try { + try { hdfsReadCheck(); } catch (IOException e) { throw new RuntimeException(e); - }*/ + } } @Test @@ -108,10 +108,8 @@ public void hdfsPruneTest() throws IOException { LOGGER.info("Path "+path+" created."); } - // TODO: Which timestamp should be used for pruning? The HDFS timestamp is easy to fetch and use but it is inaccurate especially for the first full cycle when the cfe_39 is first started. - // It would be much more heavy task to check the timestamps of the records that are stored in the file, rather than just checking the file timestamp. - - // TODO: To bypass the issue of modification timestamps being inaccurate there is the method of altering the modification timestamps to mirror the timestamp of the records during the initial commit of the file: + // TODO: Prune old records from database. + // One method is to use modification timestamp of the avro-file stored in HDFS: fs.setTimes(new Path(path+"/"+0.8), Long.parseUnsignedLong("1675930598000"), -1); // where mtime is modification time and atime is access time. -1 as input parameter leaves the original atime/mtime value as is. FileStatus[] fileStatuses = fs.listStatus(new Path(newFolderPath + "/")); for (FileStatus a : fileStatuses) { @@ -121,8 +119,19 @@ public void hdfsPruneTest() throws IOException { if (Objects.equals(a.getPath().toString(), asfas)) { Assertions.assertEquals(timestamp, new Timestamp(Long.parseUnsignedLong("1675930598000"))); LOGGER.info("testConsumerTopic/0.8 passed assertion test, timestamp was properly set to 1675930598000"); + // fs.delete(a.getPath(), true); + // If all the files have their modification timestamp altered to mirror the final record timestamp, it would be possible to prune the database based on the timestamps of the fileStatuses object. } } + // Another (most likely the best) method is to use MapReduce to prune the records in HDFS. MapReduce should allow the processing of the pruning to be efficient. + + // TODO: Query handling + // The records are in this AVRO format: + // {"timestamp": 1650872092240000, "message": "25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn audit says hi!]", "directory": "jla02logger", "stream": "test:jla02logger:0", "host": "jla-02.default", "input": "imrelp:cfe-06-0.cfe-06.default:", "partition": "8", "offset": 8, "origin": "jla-02.default"} + // Query handler must be implemented in a way that the AVRO files are first opened, then processed to syslog format and then sent to the query requester. The records are processed/filtered based on the given query conditions using MapReduce to make the code capable of processing the vast amounts of records that are expected. + // MapReduce functionalities of the Hadoop cluster: https://hadoop.apache.org/docs/stable/hadoop-mapreduce-client/hadoop-mapreduce-client-core/MapReduceTutorial.html + + fs.close(); } From d88b36ec3d70ec5033c2723f15ae0043051bed99 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Tue, 13 Feb 2024 15:54:29 +0200 Subject: [PATCH 045/146] Setting up pruning using avro-mapred. --- pom.xml | 5 ++ .../com/teragrep/cfe_39/CombinedFullTest.java | 25 ++---- .../java/com/teragrep/cfe_39/PruneTest.java | 90 +++++++++++++++++++ 3 files changed, 100 insertions(+), 20 deletions(-) create mode 100644 src/test/java/com/teragrep/cfe_39/PruneTest.java diff --git a/pom.xml b/pom.xml index a28b7920..a7d217be 100644 --- a/pom.xml +++ b/pom.xml @@ -125,6 +125,11 @@ avro 1.11.3 + + org.apache.avro + avro-mapred + 1.11.3 + org.apache.hadoop diff --git a/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java b/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java index 79c8d700..89e93790 100644 --- a/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java +++ b/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java @@ -1,7 +1,6 @@ package com.teragrep.cfe_39; import com.teragrep.cfe_39.avro.SyslogRecord; -import com.teragrep.cfe_39.consumers.kafka.DatabaseOutput; import com.teragrep.cfe_39.consumers.kafka.KafkaController; import org.apache.avro.file.DataFileStream; import org.apache.avro.specific.SpecificDatumReader; @@ -15,15 +14,14 @@ import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hadoop.fs.Path; import java.io.File; import java.io.IOException; import java.net.URI; import java.nio.file.Files; -import java.sql.Timestamp; import java.util.ArrayList; import java.util.List; -import java.util.Objects; public class CombinedFullTest { @@ -108,28 +106,15 @@ public void hdfsPruneTest() throws IOException { LOGGER.info("Path "+path+" created."); } - // TODO: Prune old records from database. - // One method is to use modification timestamp of the avro-file stored in HDFS: - fs.setTimes(new Path(path+"/"+0.8), Long.parseUnsignedLong("1675930598000"), -1); // where mtime is modification time and atime is access time. -1 as input parameter leaves the original atime/mtime value as is. - FileStatus[] fileStatuses = fs.listStatus(new Path(newFolderPath + "/")); - for (FileStatus a : fileStatuses) { - Timestamp timestamp = new Timestamp(a.getModificationTime()); - String asfas = hdfsuri + "opt/teragrep/cfe_39/srv/testConsumerTopic/" + 0.8; - LOGGER.info(asfas); - if (Objects.equals(a.getPath().toString(), asfas)) { - Assertions.assertEquals(timestamp, new Timestamp(Long.parseUnsignedLong("1675930598000"))); - LOGGER.info("testConsumerTopic/0.8 passed assertion test, timestamp was properly set to 1675930598000"); - // fs.delete(a.getPath(), true); - // If all the files have their modification timestamp altered to mirror the final record timestamp, it would be possible to prune the database based on the timestamps of the fileStatuses object. - } - } - // Another (most likely the best) method is to use MapReduce to prune the records in HDFS. MapReduce should allow the processing of the pruning to be efficient. + // TODO: Use MapReduce for pruning. AVRO and MapReduce have great compatibility and support with each other. Tests for pruning are carried out in PruneTest.java before integration testing here. - // TODO: Query handling + // TODO: Use MapReduce for Query handling. // The records are in this AVRO format: // {"timestamp": 1650872092240000, "message": "25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn audit says hi!]", "directory": "jla02logger", "stream": "test:jla02logger:0", "host": "jla-02.default", "input": "imrelp:cfe-06-0.cfe-06.default:", "partition": "8", "offset": 8, "origin": "jla-02.default"} // Query handler must be implemented in a way that the AVRO files are first opened, then processed to syslog format and then sent to the query requester. The records are processed/filtered based on the given query conditions using MapReduce to make the code capable of processing the vast amounts of records that are expected. + // MapReduce functionalities of the Hadoop cluster: https://hadoop.apache.org/docs/stable/hadoop-mapreduce-client/hadoop-mapreduce-client-core/MapReduceTutorial.html + // Avro side of documentations for MapReduce: https://avro.apache.org/docs/1.11.1/mapreduce-guide/ fs.close(); diff --git a/src/test/java/com/teragrep/cfe_39/PruneTest.java b/src/test/java/com/teragrep/cfe_39/PruneTest.java new file mode 100644 index 00000000..9a8e7c70 --- /dev/null +++ b/src/test/java/com/teragrep/cfe_39/PruneTest.java @@ -0,0 +1,90 @@ +package com.teragrep.cfe_39; + +import com.teragrep.cfe_39.avro.SyslogRecord; +import org.apache.avro.Schema; +import org.apache.avro.mapred.AvroKey; +import org.apache.avro.mapred.AvroValue; +import org.apache.avro.mapreduce.AvroJob; +import org.apache.avro.mapreduce.AvroKeyInputFormat; +import org.apache.avro.mapreduce.AvroKeyValueOutputFormat; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.conf.Configured; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.IntWritable; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.Mapper; +import org.apache.hadoop.mapreduce.Reducer; +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; +import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; +import org.apache.hadoop.util.Tool; +import org.apache.hadoop.util.ToolRunner; + +import java.io.IOException; + +// TODO: The main function that will call for pruning will know the topic name (aka. folder path). The pruning will be done in folder basis, aka. in topic basis, so tracking the topic name is not important for the MapReduce. +// Instead the partition and offset values together with timestamp are important for pruning. The function should return a list of key-value pairs where key is the partition and value is the last offset that is outside of the given timestamp limit. +// The pruning of old records can be called in KafkaController.java row 112. This way the records are pruned every time new ones are added. Make sure there are no concurrency issues with the HDFS writer. Most likely there is a need for pruning-controller class that will manage the folder/topic scanning etc. +public class PruneTest extends Configured implements Tool { + + + // TODO: Change mapper in a way that it extracts timestamp from the input SyslogRecord. + // TimestampMapper takes a SyslogRecord as input and outputs a key-value pair based on the input. + public static class TimestampMapper extends Mapper, NullWritable, Text, LongWritable> { + @Override + public void map(AvroKey key, NullWritable value, Context context) throws IOException, InterruptedException { + Long timestamp = key.datum().getTimestamp(); + CharSequence partition = key.datum().getPartition(); + long offset = key.datum().getOffset(); + context.write(new Text(partition + "." + offset), new LongWritable(timestamp)); // TODO: Maybe change the output so the key will contain both the partition and offset (partition + "." + offset), while value will contain the timestamp. This way reduce can do the pruning with the mapper output. + } + } + + + public static class TimestampReducer extends Reducer, AvroValue> { + @Override + public void reduce(Text key, Iterable values, Context context) throws IOException, InterruptedException { + // TODO: Filter the input (a list of key-value pairs where value is a timestamp) in a way that any value over X is filtered out of the key-value list. + long sum = 0; + for (LongWritable value : values) { + sum += value.get(); + } + context.write(new AvroKey(key.toString()), new AvroValue(sum)); // FIXME + } + } + + public int run(String[] args) throws Exception { + if (args.length != 2) { + System.err.println("Usage: MapReduceTimestampPrune "); + return -1; + } + + Configuration conf = new Configuration(); + Job job = Job.getInstance(conf, "timestamp prune"); + job.setJarByClass(PruneTest.class); + + FileInputFormat.setInputPaths(job, new Path(args[0])); // The input path should be the folder where the AVRO-files are held. setInputPaths can take either folder or file as input, not sure if using folder has the same effect as having a list of files. + FileOutputFormat.setOutputPath(job, new Path(args[1])); + + job.setInputFormatClass(AvroKeyInputFormat.class); + job.setMapperClass(TimestampMapper.class); + AvroJob.setInputKeySchema(job, SyslogRecord.getClassSchema()); + job.setMapOutputKeyClass(Text.class); + job.setMapOutputValueClass(LongWritable.class); + + job.setOutputFormatClass(AvroKeyValueOutputFormat.class); + job.setReducerClass(TimestampReducer.class); + AvroJob.setOutputKeySchema(job, Schema.create(Schema.Type.STRING)); + AvroJob.setOutputValueSchema(job, Schema.create(Schema.Type.LONG)); + + return (job.waitForCompletion(true) ? 0 : 1); + } + + // Set input folder to be the topic folder. + public static void main(String[] args) throws Exception { + int res = ToolRunner.run(new PruneTest(), args); // arg1 is and arg2 is , output path should be a new HDFS folder that does not exist and input path should be the HDFS folder with AVRO-files that we have generated in tests. + System.exit(res); + } +} From b0199a3a50591906a47abf1aa1fce6564646e054 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Thu, 15 Feb 2024 14:15:08 +0200 Subject: [PATCH 046/146] Setting up pruning using avro-mapred. (2) --- .../java/com/teragrep/cfe_39/PruneTest.java | 41 ++++++++++++++++--- 1 file changed, 35 insertions(+), 6 deletions(-) diff --git a/src/test/java/com/teragrep/cfe_39/PruneTest.java b/src/test/java/com/teragrep/cfe_39/PruneTest.java index 9a8e7c70..a5c2a847 100644 --- a/src/test/java/com/teragrep/cfe_39/PruneTest.java +++ b/src/test/java/com/teragrep/cfe_39/PruneTest.java @@ -24,12 +24,33 @@ import java.io.IOException; +/* +The records are stored inside files that are 64MB in size and named depending on which Kafka partition offset the last stored record belongs to. +In other words the files are inside topic_name-folder and there are at least one file per partition, depending on the load size of records that are fetched from Kafka topics. + +The AVRO-files that hold the records can house over 240,000 records each (at 64MB), but that is just an estimate because the record sizes vary widely. +This means that the partition offsets are not guaranteed to be same for all the files in a topic. In other words the same number of records could be distributed between 3 files on one partition and 2 files on another partition because of the different individual record sizes. +Lets take 2 topic partitions and their files as an example: + +topic_name/0.25 (contains partition 0 records between offsets 0 and 25) +topic_name/0.50 (contains partition 0 records between offsets 26 and 50) +topic_name/1.35 (contains partition 1 records between offsets 0 and 35) +topic_name/1.55 (contains partition 1 records between offsets 36 and 55) + +Timestamps for the record offsets are linear: record 0.1 is timed first, record 1.1 is second, 0.2 is third, 1.2 is fourth, etc. +Lets say that pruning cutoff epoch lands between records 0.30 and 1.30. This means that the file topic_name/0.25 is pruned but topic_name/1.35 is not. +topic_name/1.35 will contain records from between offsets 0 and 35. Only the records at or above offset 30 are coherent, everything under that can be considered to be garbage data if they are ever queried. + +Should the file be altered so the garbage data can be removed from the file? Or should the records only be filtered out when responding to queries with result sets? +The filtering method is most likely the least resource intensive method as it can be done during the MapReduce, and the amount of garbage record shouldn't be too much. If the main function decides on the cutoff epoch, then it can also track it for the datasource function to use for filtering. +But in any case the pruning should include deleting AVRO-files that hold only outdated records that should be pruned. The handling of the leftover garbage records can be handled later. +*/ + // TODO: The main function that will call for pruning will know the topic name (aka. folder path). The pruning will be done in folder basis, aka. in topic basis, so tracking the topic name is not important for the MapReduce. -// Instead the partition and offset values together with timestamp are important for pruning. The function should return a list of key-value pairs where key is the partition and value is the last offset that is outside of the given timestamp limit. +// Instead, the partition and offset values together with timestamp are important for pruning. The function should return a list of key-value pairs where key is the partition and value is the last offset that is outside of the given timestamp limit. // The pruning of old records can be called in KafkaController.java row 112. This way the records are pruned every time new ones are added. Make sure there are no concurrency issues with the HDFS writer. Most likely there is a need for pruning-controller class that will manage the folder/topic scanning etc. public class PruneTest extends Configured implements Tool { - - + long cutoff_epoch = 0L; // TODO: Change mapper in a way that it extracts timestamp from the input SyslogRecord. // TimestampMapper takes a SyslogRecord as input and outputs a key-value pair based on the input. public static class TimestampMapper extends Mapper, NullWritable, Text, LongWritable> { @@ -38,7 +59,7 @@ public void map(AvroKey key, NullWritable value, Context context) Long timestamp = key.datum().getTimestamp(); CharSequence partition = key.datum().getPartition(); long offset = key.datum().getOffset(); - context.write(new Text(partition + "." + offset), new LongWritable(timestamp)); // TODO: Maybe change the output so the key will contain both the partition and offset (partition + "." + offset), while value will contain the timestamp. This way reduce can do the pruning with the mapper output. + context.write(new Text(partition + "." + offset), new LongWritable(timestamp)); // Changed the output so the key will contain both the partition and offset (partition + "." + offset), while value will contain the timestamp. This way reduce can do the pruning with the mapper output. } } @@ -56,8 +77,14 @@ public void reduce(Text key, Iterable values, Context context) thr } public int run(String[] args) throws Exception { - if (args.length != 2) { - System.err.println("Usage: MapReduceTimestampPrune "); + if (args.length != 3) { + System.err.println("Usage: MapReduceTimestampPrune "); + return -1; + } + try { + cutoff_epoch = Long.parseLong(args[2]); + } catch (NumberFormatException e) { + System.err.println("Usage: 3rd input argument should be parseable to long"); return -1; } @@ -82,6 +109,8 @@ public int run(String[] args) throws Exception { return (job.waitForCompletion(true) ? 0 : 1); } + // TODO: Should there be more input parameters? Like topic name, a list of file names that hold the records, etc? That would open up a lot more leeway for coding the pruning algorithm using MapReduce. + // Set input folder to be the topic folder. public static void main(String[] args) throws Exception { int res = ToolRunner.run(new PruneTest(), args); // arg1 is and arg2 is , output path should be a new HDFS folder that does not exist and input path should be the HDFS folder with AVRO-files that we have generated in tests. From c9f330bf086a561e8a7250fd01aa099b99caf4d4 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Thu, 15 Feb 2024 16:57:20 +0200 Subject: [PATCH 047/146] Setting up pruning using avro-mapred. (3) --- .../java/com/teragrep/cfe_39/PruneTest.java | 30 +++++++++---------- 1 file changed, 15 insertions(+), 15 deletions(-) diff --git a/src/test/java/com/teragrep/cfe_39/PruneTest.java b/src/test/java/com/teragrep/cfe_39/PruneTest.java index a5c2a847..35c91296 100644 --- a/src/test/java/com/teragrep/cfe_39/PruneTest.java +++ b/src/test/java/com/teragrep/cfe_39/PruneTest.java @@ -24,6 +24,8 @@ import java.io.IOException; +// TODO: This class should be compiled into a jar-file that is then sent to the hadoop cluster for running the job when needed. Maven should be configured to do the jar-packaging etc.. + /* The records are stored inside files that are 64MB in size and named depending on which Kafka partition offset the last stored record belongs to. In other words the files are inside topic_name-folder and there are at least one file per partition, depending on the load size of records that are fetched from Kafka topics. @@ -43,20 +45,19 @@ Should the file be altered so the garbage data can be removed from the file? Or should the records only be filtered out when responding to queries with result sets? The filtering method is most likely the least resource intensive method as it can be done during the MapReduce, and the amount of garbage record shouldn't be too much. If the main function decides on the cutoff epoch, then it can also track it for the datasource function to use for filtering. -But in any case the pruning should include deleting AVRO-files that hold only outdated records that should be pruned. The handling of the leftover garbage records can be handled later. +But in any case the pruning should include deleting AVRO-files that hold only outdated records that should be pruned. The handling of the leftover garbage records can be handled later in the MapReduce of the datasource component queries. */ -// TODO: The main function that will call for pruning will know the topic name (aka. folder path). The pruning will be done in folder basis, aka. in topic basis, so tracking the topic name is not important for the MapReduce. -// Instead, the partition and offset values together with timestamp are important for pruning. The function should return a list of key-value pairs where key is the partition and value is the last offset that is outside of the given timestamp limit. -// The pruning of old records can be called in KafkaController.java row 112. This way the records are pruned every time new ones are added. Make sure there are no concurrency issues with the HDFS writer. Most likely there is a need for pruning-controller class that will manage the folder/topic scanning etc. +// The main function that will call for pruning will know the topic name (aka. folder path). The pruning will be done in folder basis, aka. in topic basis, so tracking the topic name is not important for the MapReduce as the input path already contains the topic name. +// Instead, the partition and offset values together with timestamp are important for pruning. The MapReduce function should create a list of key-value pairs where key is the partition+offset and value is the timestamp, where timestamp is smaller than the cutoff_epoch defined by input arguments. +// The pruning of old records can be called in KafkaController.java row 112. This way the records are pruned every time new ones are added. Make sure there are no concurrency issues with the HDFS writer. Most likely there is a need for pruning-controller class that will manage the folder/topic scanning etc. public class PruneTest extends Configured implements Tool { - long cutoff_epoch = 0L; - // TODO: Change mapper in a way that it extracts timestamp from the input SyslogRecord. - // TimestampMapper takes a SyslogRecord as input and outputs a key-value pair based on the input. + static long cutoff_epoch; + // TimestampMapper takes a SyslogRecord as input and outputs a key-value pair of record partition+"."+offset and timestamp of the record. public static class TimestampMapper extends Mapper, NullWritable, Text, LongWritable> { @Override public void map(AvroKey key, NullWritable value, Context context) throws IOException, InterruptedException { - Long timestamp = key.datum().getTimestamp(); + long timestamp = key.datum().getTimestamp(); CharSequence partition = key.datum().getPartition(); long offset = key.datum().getOffset(); context.write(new Text(partition + "." + offset), new LongWritable(timestamp)); // Changed the output so the key will contain both the partition and offset (partition + "." + offset), while value will contain the timestamp. This way reduce can do the pruning with the mapper output. @@ -64,15 +65,16 @@ public void map(AvroKey key, NullWritable value, Context context) } - public static class TimestampReducer extends Reducer, AvroValue> { + public static class TimestampReducer extends Reducer, AvroValue> { @Override public void reduce(Text key, Iterable values, Context context) throws IOException, InterruptedException { - // TODO: Filter the input (a list of key-value pairs where value is a timestamp) in a way that any value over X is filtered out of the key-value list. - long sum = 0; + long timestamp; for (LongWritable value : values) { - sum += value.get(); + timestamp = value.get(); + if (timestamp < cutoff_epoch) { // TODO: inclusive or exclusive? Exclusive for now. + context.write(new AvroKey(key.toString()), new AvroValue(timestamp)); + } } - context.write(new AvroKey(key.toString()), new AvroValue(sum)); // FIXME } } @@ -109,8 +111,6 @@ public int run(String[] args) throws Exception { return (job.waitForCompletion(true) ? 0 : 1); } - // TODO: Should there be more input parameters? Like topic name, a list of file names that hold the records, etc? That would open up a lot more leeway for coding the pruning algorithm using MapReduce. - // Set input folder to be the topic folder. public static void main(String[] args) throws Exception { int res = ToolRunner.run(new PruneTest(), args); // arg1 is and arg2 is , output path should be a new HDFS folder that does not exist and input path should be the HDFS folder with AVRO-files that we have generated in tests. From 832ae6a9bd2e1864e1178b6c392fa5938b96d509 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Fri, 16 Feb 2024 16:01:52 +0200 Subject: [PATCH 048/146] Attempt at optimizing the output of MapReduce. --- src/test/java/com/teragrep/cfe_39/PruneTest.java | 16 +++++++++++++--- 1 file changed, 13 insertions(+), 3 deletions(-) diff --git a/src/test/java/com/teragrep/cfe_39/PruneTest.java b/src/test/java/com/teragrep/cfe_39/PruneTest.java index 35c91296..6b79c915 100644 --- a/src/test/java/com/teragrep/cfe_39/PruneTest.java +++ b/src/test/java/com/teragrep/cfe_39/PruneTest.java @@ -14,17 +14,20 @@ import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapreduce.InputSplit; import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.Mapper; import org.apache.hadoop.mapreduce.Reducer; import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; +import org.apache.hadoop.mapreduce.lib.input.FileSplit; +import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat; import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.ToolRunner; import java.io.IOException; -// TODO: This class should be compiled into a jar-file that is then sent to the hadoop cluster for running the job when needed. Maven should be configured to do the jar-packaging etc.. +// TODO: This class should be compiled into a jar-file that is then sent to the hadoop cluster for running the job when needed. Maven should be configured to do the jar-packaging etc. /* The records are stored inside files that are 64MB in size and named depending on which Kafka partition offset the last stored record belongs to. @@ -57,6 +60,12 @@ public class PruneTest extends Configured implements Tool { public static class TimestampMapper extends Mapper, NullWritable, Text, LongWritable> { @Override public void map(AvroKey key, NullWritable value, Context context) throws IOException, InterruptedException { + + // TODO: Add context.getInputSplit(); functionality to the mapper that allows tracking of the filenames that the records originate from. + // FIXME: Casting context.getInputSplit()).getPath() to FileSplit shouldn't work anymore with newer versions of hadoop. Input split class now returns TaggedInputSplit instead. + String filename = ((FileSplit) context.getInputSplit()).getPath().getName(); + // If the FileSplit casting starts working for some reason on the newest version of MapReduce, it can be used to filter out only the specific filenames that should be deleted from HDFS. This way the size of the MapReduce output will be optimized and doesn't need any additional processing by the client. + long timestamp = key.datum().getTimestamp(); CharSequence partition = key.datum().getPartition(); long offset = key.datum().getOffset(); @@ -65,6 +74,7 @@ public void map(AvroKey key, NullWritable value, Context context) } + // Removes all the key-timestamp pairs that have timestamp over the cutoff_epoch. What is left are list of keys (outdated records) that should be removed from the HDFS database. public static class TimestampReducer extends Reducer, AvroValue> { @Override public void reduce(Text key, Iterable values, Context context) throws IOException, InterruptedException { @@ -94,8 +104,8 @@ public int run(String[] args) throws Exception { Job job = Job.getInstance(conf, "timestamp prune"); job.setJarByClass(PruneTest.class); - FileInputFormat.setInputPaths(job, new Path(args[0])); // The input path should be the folder where the AVRO-files are held. setInputPaths can take either folder or file as input, not sure if using folder has the same effect as having a list of files. - FileOutputFormat.setOutputPath(job, new Path(args[1])); + SequenceFileInputFormat.setInputPaths(job, new Path(args[0])); // The input path should be the folder where the AVRO-files are held. setInputPaths can take either folder or file as input, not sure if using folder has the same effect as having a list of files. + FileOutputFormat.setOutputPath(job, new Path(args[1])); // Output path is where the results of the MapReduce are stored. job.setInputFormatClass(AvroKeyInputFormat.class); job.setMapperClass(TimestampMapper.class); From 6af8eb49d39a2d41201ac7da19ac1c96f238555e Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Mon, 19 Feb 2024 15:55:27 +0200 Subject: [PATCH 049/146] Reverting back to using FileSystem modification timestamps for pruning instead of MapReduce. Implemented HDFSPrune.java which prunes the HDFS database. --- src/main/java/com/teragrep/cfe_39/Config.java | 1 + .../consumers/kafka/DatabaseOutput.java | 14 +++- .../cfe_39/consumers/kafka/HDFSPrune.java | 82 +++++++++++++++++++ .../cfe_39/consumers/kafka/HDFSWriter.java | 41 +++++++--- .../consumers/kafka/KafkaController.java | 13 ++- .../com/teragrep/cfe_39/CombinedFullTest.java | 35 ++++++-- .../java/com/teragrep/cfe_39/HdfsTest.java | 2 +- .../java/com/teragrep/cfe_39/PruneTest.java | 5 +- 8 files changed, 168 insertions(+), 25 deletions(-) create mode 100644 src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSPrune.java diff --git a/src/main/java/com/teragrep/cfe_39/Config.java b/src/main/java/com/teragrep/cfe_39/Config.java index 182da80a..b4bccea8 100644 --- a/src/main/java/com/teragrep/cfe_39/Config.java +++ b/src/main/java/com/teragrep/cfe_39/Config.java @@ -66,6 +66,7 @@ public class Config { private final int numOfConsumers; // TODO: Set up configuration check for important parameters. Remove old unused parameters. + // TODO: Implement cutoff_offset parameter get/set for pruning. Config() throws IOException { Properties properties = new Properties(); diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java index 871c9a6b..a4415342 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java @@ -72,6 +72,7 @@ public class DatabaseOutput implements Consumer> { private File syslogFile; private final Config config; private long approximatedSize; // FIXME: Not working properly when flush() is not used after append in the AVRO-file. + private long epochMicros_last; DatabaseOutput( Config config, @@ -102,6 +103,7 @@ public class DatabaseOutput implements Consumer> { this.eventNodeRelayHostname = new SDVector("event_node_relay@48577","hostname"); this.originHostname = new SDVector("origin@48577","hostname"); this.approximatedSize = 0; + this.epochMicros_last = 0L; } // Checks that the filesize stays under the defined maximum file size. @@ -114,7 +116,7 @@ boolean checkSizeTooLarge(long fileSize, RecordOffsetObject recordOffsetObject) // This part closes the writing of now "complete" AVRO-file and stores the file to HDFS. syslogAvroWriter.close(); try (HDFSWriter writer = new HDFSWriter(config, recordOffsetObject)) { - writer.commit(syslogFile); // commits the final AVRO-file to HDFS. + writer.commit(syslogFile, epochMicros_last); // commits the final AVRO-file to HDFS. } // This part defines a new empty file to which the new AVRO-serialized records are stored until it again hits the 64M size limit. @@ -186,7 +188,7 @@ public void accept(List recordOffsetObjectList) { // Previous partition was fully consumed. Commit file to HDFS and create a new AVRO-file. syslogAvroWriter.close(); try (HDFSWriter writer = new HDFSWriter(config, lastObject)) { - writer.commit(syslogFile); + writer.commit(syslogFile, epochMicros_last); } // This part defines a new empty file to which the new AVRO-serialized records are stored until it again hits the 64M size limit. @@ -249,9 +251,15 @@ public void accept(List recordOffsetObjectList) { // Calculate the size of syslogRecord that is going to be written to syslogAvroWriter-file. long capacity = syslogRecord.toByteBuffer().capacity(); + // handle initial timestamp + if (epochMicros_last == 0L) { + epochMicros_last = epochMicros; + } // Check if there is still room in syslogAvroWriter for another syslogRecord. Commit syslogAvroWriter to HDFS if no room left, emptying it out in the process. // checkSizeTooLarge(approximatedSize + capacity, lastObject); // FIXME: approximatedSize is not working properly without the use of flush() after append. File sizes are all over the place. checkSizeTooLarge(syslogAvroWriter.getFileSize() + capacity, lastObject); + // if more records can be inserted, update epochMicros_last with the timestamp of the last inserted record. + epochMicros_last = epochMicros; // Add syslogRecord to syslogAvroWriter which has room for new syslogRecord. syslogAvroWriter.write(syslogRecord); approximatedSize += capacity; @@ -268,7 +276,7 @@ public void accept(List recordOffsetObjectList) { if (syslogAvroWriter != null) { syslogAvroWriter.close(); try (HDFSWriter writer = new HDFSWriter(config, lastObject)) { - writer.commit(syslogFile); // commits the final AVRO-file to HDFS. + writer.commit(syslogFile, epochMicros_last); // commits the final AVRO-file to HDFS. } } } catch (IOException e) { diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSPrune.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSPrune.java new file mode 100644 index 00000000..541cc1b0 --- /dev/null +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSPrune.java @@ -0,0 +1,82 @@ +/* + HDFS Data Ingestion for PTH_06 use CFE-39 + Copyright (C) 2022 Fail-Safe IT Solutions Oy + + 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. + */ + +package com.teragrep.cfe_39.consumers.kafka; + +import com.teragrep.cfe_39.Config; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.net.URI; +import java.util.concurrent.TimeUnit; + +public class HDFSPrune { + private static final Logger LOGGER = LoggerFactory.getLogger(HDFSPrune.class); + private Config config; + private final FileSystem fs; + private Path newFolderPath; + private long cutoff_epoch; + + public HDFSPrune(Config config, String topicName) throws IOException { + this.config = config; + String hdfsuri = config.getHdfsuri(); + + String path = config.getHdfsPath()+"/"+topicName; + // ====== Init HDFS File System Object + Configuration conf = new Configuration(); + // Set FileSystem URI + conf.set("fs.defaultFS", hdfsuri); + // Because of Maven + conf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); + conf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); + // Set HADOOP user + System.setProperty("HADOOP_USER_NAME", "hdfs"); + System.setProperty("hadoop.home.dir", "/"); + //Get the filesystem - HDFS + fs = FileSystem.get(URI.create(hdfsuri), conf); + + //==== Create folder if not exists + Path workingDir=fs.getWorkingDirectory(); + newFolderPath= new Path(path); + if(!fs.exists(newFolderPath)) { + // Create new Directory + fs.mkdirs(newFolderPath); + LOGGER.info("Path "+path+" created."); + } + + cutoff_epoch = System.currentTimeMillis() - 172800000L; // TODO: cutoff offset is 172800000L only for testing, parametrize it using Config.java. + } + + public void prune() throws IOException { + // Fetch the filestatuses of HDFS files. + FileStatus[] fileStatuses = fs.listStatus(new Path(newFolderPath + "/")); + for (FileStatus a : fileStatuses) { + // If all the files have their modification timestamp altered to mirror the final record timestamp, it is possible to prune the database based on the timestamps of the fileStatuses object. + long convert = TimeUnit.MILLISECONDS.convert(a.getModificationTime(), TimeUnit.MICROSECONDS); // MICROSECONDS ARE NOT SUPPORTED, convert the microsecond epoch to milliseconds. + // Delete old files + if (convert < cutoff_epoch) { + boolean delete = fs.delete(a.getPath(), true); + LOGGER.info("Deleted file " + a.getPath()); + } + } + } +} diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java index f199b075..815c8675 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java @@ -39,6 +39,8 @@ public class HDFSWriter implements AutoCloseable{ private final String path; private final FileSystem fs; private final boolean useMockKafkaConsumer; // test-mode switch + private final Configuration conf; + private final String hdfsuri; // Create files as whole but stream the contents into them. Avro files 'flush' must be called as few times as possible. Check memory usage impact // Later make sure to check the avro file flush issue where the file size is all over the place if flush is not used after every append to the file. @@ -53,7 +55,7 @@ public HDFSWriter(Config config, RecordOffsetObject lastObject) throws IOExcepti if (useMockKafkaConsumer) { // Code for initializing the class in test mode without kerberos. - String hdfsuri = config.getHdfsuri(); // Get from config. + hdfsuri = config.getHdfsuri(); // Get from config. // The filepath should be something like hdfs:///opt/teragrep/cfe_39/srv/topic_name/0.12345 where 12345 is offset and 0 the partition. // In other words the folder named topic_name holds files that are named and arranged based on partition and the partition's offset. Every partition has its own set of unique offset values. @@ -62,7 +64,7 @@ public HDFSWriter(Config config, RecordOffsetObject lastObject) throws IOExcepti fileName = lastObject.partition+"."+lastObject.offset; // filename should be constructed from partition and offset. // ====== Init HDFS File System Object - Configuration conf = new Configuration(); + conf = new Configuration(); // Set FileSystem URI conf.set("fs.defaultFS", hdfsuri); // Because of Maven @@ -81,7 +83,7 @@ public HDFSWriter(Config config, RecordOffsetObject lastObject) throws IOExcepti }else { // Code for initializing the class with kerberos. - String hdfsuri = config.getHdfsuri(); // Get from config. + hdfsuri = config.getHdfsuri(); // Get from config. // The filepath should be something like hdfs:///opt/teragrep/cfe_39/srv/topic_name/0.12345 where 12345 is offset and 0 the partition. // In other words the folder named topic_name holds files that are named and arranged based on partition and the partition's offset. Every partition has its own set of unique offset values. @@ -98,7 +100,7 @@ public HDFSWriter(Config config, RecordOffsetObject lastObject) throws IOExcepti System.setProperty("java.security.krb5.realm", config.getKerberosRealm()); System.setProperty("java.security.krb5.kdc", config.getKerberosHost()); - Configuration conf = new Configuration(); + conf = new Configuration(); // enable kerberus conf.set("hadoop.security.authentication", config.getHadoopAuthentication()); @@ -126,7 +128,7 @@ public HDFSWriter(Config config, RecordOffsetObject lastObject) throws IOExcepti } // Method for committing the AVRO-file to HDFS - public void commit(File syslogFile) { + public void commit(File syslogFile, long epochMicros_last) { // The code for writing the file to HDFS should be same for both test (non-kerberized access) and prod (kerberized access). if (useMockKafkaConsumer) { // CODE FOR TEST-MODE GOES HERE! @@ -150,19 +152,23 @@ public void commit(File syslogFile) { throw new RuntimeException("File " + fileName + " already exists"); } - //Init output stream + /*//Init output stream FSDataOutputStream outputStream = fs.create(hdfswritepath); // Write the file contents of syslogFile to hdfswritepath in HDFS. // file to bytes[] - /*byte[] bytearray = new byte[(int) syslogFile.length()]; + *//*byte[] bytearray = new byte[(int) syslogFile.length()]; try (FileInputStream inputStream = new FileInputStream(syslogFile)) { inputStream.read(bytearray); - }*/ + }*//* byte[] bytes = Files.readAllBytes(Paths.get(syslogFile.getPath())); // if readAllBytes is not efficient use FileInputStream outputStream.write(bytes); + outputStream.close();*/ - outputStream.close(); + Path path = new Path(syslogFile.getPath()); + fs.copyFromLocalFile(path, hdfswritepath); + fs.setTimes(hdfswritepath, epochMicros_last, -1); // where mtime is modification time and atime is access time. -1 as input parameter leaves the original atime/mtime value as is. + // updateTimestamp(hdfswritepath, epochMicros_last); LOGGER.debug("End Write file into hdfs"); boolean delete = syslogFile.delete(); // deletes the avro-file from the local disk now that it has been committed to HDFS. LOGGER.debug("\n" + "File committed to HDFS, file writepath should be: " + hdfswritepath.toString() + "\n"); @@ -214,15 +220,28 @@ public void commit(File syslogFile) { } } + private void updateTimestamp(Path hdfswritepath, long epochMicros_last) { + // Testing timestamp editing. The new timestamp should be the timestamp of the last record that was added to the AVRO-file. + try { + FileSystem fs_temp = FileSystem.get(URI.create(hdfsuri), conf); + FSDataOutputStream fsDataOutputStream = fs_temp.create(hdfswritepath); + fs_temp.setTimes(hdfswritepath, epochMicros_last, -1); // where mtime is modification time and atime is access time. -1 as input parameter leaves the original atime/mtime value as is. + fsDataOutputStream.close(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + // try-with-resources handles closing the filesystem automatically. public void close() { - if (fs != null) { + // FIXME: fs.close() doesn't just affect the current class, it affects all the FileSystem objects that were created using FileSystem.get(URI.create(hdfsuri), conf); in different threads. + /*if (fs != null) { try { fs.close(); } catch (IOException e) { throw new RuntimeException(e); } - } + }*/ } diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java index f175c4cb..37243c15 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java @@ -27,6 +27,7 @@ import org.slf4j.LoggerFactory; +import java.io.IOException; import java.sql.SQLException; import java.time.Duration; import java.util.*; @@ -103,7 +104,17 @@ public void run() throws InterruptedException { durationStatistics.log(); long topicScanDelay = 30000L; Thread.sleep(topicScanDelay); - + for (String topic_name : activeTopics) { + LOGGER.info("topic that is being bruned: " + topic_name); + if (topic_name != null) { + try { + HDFSPrune hdfsPrune = new HDFSPrune(config, topic_name); + hdfsPrune.prune(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + } // For testing purposes only. Stops the run when all the records are consumed from the mockConsumer during test. if (durationStatistics.getTotalRecords() > 0 & useMockKafkaConsumer) { LOGGER.debug("Processed all the test records. Closing."); diff --git a/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java b/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java index 89e93790..a4491929 100644 --- a/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java +++ b/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java @@ -20,8 +20,9 @@ import java.io.IOException; import java.net.URI; import java.nio.file.Files; -import java.util.ArrayList; -import java.util.List; +import java.sql.Timestamp; +import java.util.*; +import java.util.concurrent.TimeUnit; public class CombinedFullTest { @@ -71,11 +72,12 @@ public void kafkaAndAvroFullTest() throws InterruptedException { kafkaController.run(); // The avro files should be committed to HDFS now. Check the committed files for any errors. // There should be 20 files, 10 partitions with each having 2 files assigned to them. - try { + // TODO: hdfsReadCheck(); does not work properly if pruning is enabled. Add checks for pruning etc. + /*try { hdfsReadCheck(); } catch (IOException e) { throw new RuntimeException(e); - } + }*/ } @Test @@ -106,17 +108,36 @@ public void hdfsPruneTest() throws IOException { LOGGER.info("Path "+path+" created."); } - // TODO: Use MapReduce for pruning. AVRO and MapReduce have great compatibility and support with each other. Tests for pruning are carried out in PruneTest.java before integration testing here. + // TODO: Use either HDFS-file modification timestamps or avro-mapred for pruning. - // TODO: Use MapReduce for Query handling. // The records are in this AVRO format: // {"timestamp": 1650872092240000, "message": "25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn audit says hi!]", "directory": "jla02logger", "stream": "test:jla02logger:0", "host": "jla-02.default", "input": "imrelp:cfe-06-0.cfe-06.default:", "partition": "8", "offset": 8, "origin": "jla-02.default"} // Query handler must be implemented in a way that the AVRO files are first opened, then processed to syslog format and then sent to the query requester. The records are processed/filtered based on the given query conditions using MapReduce to make the code capable of processing the vast amounts of records that are expected. - // MapReduce functionalities of the Hadoop cluster: https://hadoop.apache.org/docs/stable/hadoop-mapreduce-client/hadoop-mapreduce-client-core/MapReduceTutorial.html // Avro side of documentations for MapReduce: https://avro.apache.org/docs/1.11.1/mapreduce-guide/ + + // Another method for pruning aside using avro-mapred is to use modification timestamp of the avro-file stored in HDFS: + // fs.setTimes(new Path(path+"/"+0.8), Long.parseUnsignedLong("1675930598000"), -1); + // where mtime is modification time and atime is access time. -1 as input parameter leaves the original atime/mtime value as is. + FileStatus[] fileStatuses = fs.listStatus(new Path(newFolderPath + "/")); + long count = Arrays.stream(fileStatuses).count(); + if (count != 0) { + for (FileStatus a : fileStatuses) { + // If all the files have their modification timestamp altered to mirror the final record timestamp, it is possible to prune the database based on the timestamps of the fileStatuses object. + long convert = TimeUnit.MILLISECONDS.convert(a.getModificationTime(), TimeUnit.MICROSECONDS); // MICROSECONDS ARE NOT SUPPORTED, convert the microsecond epoch to milliseconds. + // Delete old files + if (convert < 1708343921000L) { + boolean delete = fs.delete(a.getPath(), true); + Assertions.assertTrue(delete); + LOGGER.info("Deleted file " + a.getPath()); + } + } + }else { + LOGGER.info("No files available!"); + } + fs.close(); } diff --git a/src/test/java/com/teragrep/cfe_39/HdfsTest.java b/src/test/java/com/teragrep/cfe_39/HdfsTest.java index 8e8a539d..bfbc831d 100644 --- a/src/test/java/com/teragrep/cfe_39/HdfsTest.java +++ b/src/test/java/com/teragrep/cfe_39/HdfsTest.java @@ -139,7 +139,7 @@ public void hdfsWriteTest() { RecordOffsetObject lastObject = new RecordOffsetObject("testConsumerTopic", Integer.parseInt(lastRecord.getPartition().toString()), lastRecord.getOffset(), null); // Fetch input parameters from the lastRecord SyslogRecord-object. LOGGER.debug("\n"+"Last record in the " + syslogFile.getName() + " file:" + "\ntopic: " + lastObject.topic + "\npartition: " + lastObject.partition + "\noffset: " + lastObject.offset); try (HDFSWriter writer = new HDFSWriter(config, lastObject)) { - writer.commit(syslogFile); // commits the final AVRO-file to HDFS. + writer.commit(syslogFile, -1L); // commits the final AVRO-file to HDFS. } catch (IOException e) { throw new RuntimeException(e); } diff --git a/src/test/java/com/teragrep/cfe_39/PruneTest.java b/src/test/java/com/teragrep/cfe_39/PruneTest.java index 6b79c915..508cb1c3 100644 --- a/src/test/java/com/teragrep/cfe_39/PruneTest.java +++ b/src/test/java/com/teragrep/cfe_39/PruneTest.java @@ -27,8 +27,9 @@ import java.io.IOException; -// TODO: This class should be compiled into a jar-file that is then sent to the hadoop cluster for running the job when needed. Maven should be configured to do the jar-packaging etc. +// Shelved for now. Using HDFS modification timestamps instead of avro-mapred for pruning the files. +// This class should be compiled into a jar-file that is then sent to the hadoop cluster for running the job when needed. Maven should be configured to do the jar-packaging etc. /* The records are stored inside files that are 64MB in size and named depending on which Kafka partition offset the last stored record belongs to. In other words the files are inside topic_name-folder and there are at least one file per partition, depending on the load size of records that are fetched from Kafka topics. @@ -53,7 +54,7 @@ // The main function that will call for pruning will know the topic name (aka. folder path). The pruning will be done in folder basis, aka. in topic basis, so tracking the topic name is not important for the MapReduce as the input path already contains the topic name. // Instead, the partition and offset values together with timestamp are important for pruning. The MapReduce function should create a list of key-value pairs where key is the partition+offset and value is the timestamp, where timestamp is smaller than the cutoff_epoch defined by input arguments. -// The pruning of old records can be called in KafkaController.java row 112. This way the records are pruned every time new ones are added. Make sure there are no concurrency issues with the HDFS writer. Most likely there is a need for pruning-controller class that will manage the folder/topic scanning etc. +// The pruning of old records can be called in KafkaController.java row 112, using the activeTopics list as a input argument for topic names. This way the records are pruned every time new ones are added. Make sure there are no concurrency issues with the HDFS writer. Most likely there is a need for pruning-controller class that will manage the folder/topic scanning etc. public class PruneTest extends Configured implements Tool { static long cutoff_epoch; // TimestampMapper takes a SyslogRecord as input and outputs a key-value pair of record partition+"."+offset and timestamp of the record. From 795c488666a7b34ab0d2ad5e8778187d6e7c4f77 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Tue, 20 Feb 2024 12:41:26 +0200 Subject: [PATCH 050/146] Parametrized the pruning cutoff epoch offset. Removed obsolete parameters from Config.java. Fixed CombinedFullTest.java to mirror the changes made to the code. Fixed AVRO maximumFileSize parametrization. --- src/main/java/com/teragrep/cfe_39/Config.java | 103 ++---------------- .../cfe_39/consumers/kafka/HDFSPrune.java | 4 +- .../com/teragrep/cfe_39/CombinedFullTest.java | 33 ++++-- .../teragrep/cfe_39/application.properties | 27 ++--- 4 files changed, 42 insertions(+), 125 deletions(-) diff --git a/src/main/java/com/teragrep/cfe_39/Config.java b/src/main/java/com/teragrep/cfe_39/Config.java index b4bccea8..a8692da9 100644 --- a/src/main/java/com/teragrep/cfe_39/Config.java +++ b/src/main/java/com/teragrep/cfe_39/Config.java @@ -31,25 +31,10 @@ import java.util.Properties; public class Config { - -/* // db - private final String dbConnectionUrl; - private final String dbUsername; - private final String dbPassword;*/ - // kafka private final String queueTopicPattern; - private final Properties kafkaConsumerProperties; - - private final boolean replicationEnabled; - private final int streamSize; - private final String streamUnits; private static final Logger LOGGER = LoggerFactory.getLogger(Config.class); - private final int dropPartitionsOlderThanHours; - private final int createPartitionsInAdvanceHours; - private final boolean overrideTableLocation; - private final String tableLocation; private final String hdfsPath; private String hdfsuri; private final String queueDirectory; @@ -64,9 +49,9 @@ public class Config { private final String kerberosTestMode; private long maximumFileSize; private final int numOfConsumers; + private final long prune_offset; // TODO: Set up configuration check for important parameters. Remove old unused parameters. - // TODO: Implement cutoff_offset parameter get/set for pruning. Config() throws IOException { Properties properties = new Properties(); @@ -75,49 +60,17 @@ public class Config { properties.load(Files.newInputStream(configPath)); LOGGER.debug("Got configuration: " + properties); - /*// db - this.dbConnectionUrl = properties.getProperty("db.connectionUrl"); - if (this.dbConnectionUrl == null) { - throw new IllegalArgumentException("db.connectionUrl not set"); - } - this.dbUsername = properties.getProperty("db.username"); - if (this.dbUsername == null) { - throw new IllegalArgumentException("db.username not set"); - } - this.dbPassword = properties.getProperty("db.password"); - if (this.dbPassword == null) { - throw new IllegalArgumentException("db.password not set"); - }*/ - - String replicationEnabledString = properties.getProperty("db.replicationEnabled", "false"); - this.replicationEnabled = Boolean.parseBoolean(replicationEnabledString); - - String streamBytesString = properties.getProperty("db.streamSize", "512"); - this.streamSize = Integer.parseInt(streamBytesString); - this.streamUnits = properties.getProperty("db.streamUnits", "rows"); - this.dropPartitionsOlderThanHours = Integer.parseInt(properties.getProperty("db.dropPartitionsOlderThanHours", "4")); - if(dropPartitionsOlderThanHours <= 0) { - throw new IllegalArgumentException("db.dropPartitionsOlderThanHours must be set to >0, got " + dropPartitionsOlderThanHours); - } - this.createPartitionsInAdvanceHours = Integer.parseInt(properties.getProperty("db.createPartitionsInAdvanceHours", "8")); - if(createPartitionsInAdvanceHours <= 0) { - throw new IllegalArgumentException("createPartitionsInAdvanceHours must be set to >0, got " + createPartitionsInAdvanceHours); - } - - String overrideTableLocationString = properties.getProperty("db.overrideTableLocation", "false"); - this.overrideTableLocation = Boolean.parseBoolean(overrideTableLocationString); - this.tableLocation = properties.getProperty("db.tableLocation"); - if(overrideTableLocation && tableLocation.isEmpty()) { - throw new IllegalArgumentException("db.tableLocation resulted in empty string when db.overrideTableLocation was true"); - } - // HDFS this.hdfsPath = properties.getProperty("hdfsPath", "hdfs:///opt/teragrep/cfe_39/srv/"); this.hdfsuri = properties.getProperty("hdfsuri", "hdfs://localhost:45937/"); + // HDFS pruning + this.prune_offset = Long.parseLong(properties.getProperty("prune_offset", "172800000")); + // AVRO this.queueDirectory = properties.getProperty("queueDirectory", ""); this.queueNamePrefix = properties.getProperty("queueNamePrefix", ""); + this.maximumFileSize = Long.parseLong(properties.getProperty("maximumFileSize", "60800000")); // kerberos this.kerberosHost = properties.getProperty("java.security.krb5.kdc", ""); @@ -143,7 +96,6 @@ public class Config { throw new IOException("File '" + loginConfig + "' set by java.security.auth.login.config does not exist"); } System.setProperty("java.security.auth.login.config", loginConfig); - this.maximumFileSize = 60800000; // default value // Just for loggers to work Path log4j2Config = Paths.get(properties.getProperty("log4j2.configurationFile", System.getProperty("user.dir") + "/etc/log4j2.properties")); @@ -166,26 +118,6 @@ private Properties loadSubProperties(Properties properties, String prefix) { return subProperties; } - public boolean isOverrideTableLocation() { - return overrideTableLocation; - } - - public String getTableLocation() { - return tableLocation; - } - -/* public String getDbConnectionUrl() { - return dbConnectionUrl; - } - - public String getDbUsername() { - return dbUsername; - } - - public String getDbPassword() { - return dbPassword; - }*/ - public String getHdfsPath() { return hdfsPath; } @@ -202,34 +134,12 @@ public String getQueueDirectory() { public String getQueueNamePrefix() { return queueNamePrefix; } - - public boolean isReplicationEnabled() { - return replicationEnabled; - } - - public int getStreamSize() { - return streamSize; - } - - public String getStreamUnits() { - return streamUnits; - } - public String getQueueTopicPattern() { return queueTopicPattern; } - public Properties getKafkaConsumerProperties() { return kafkaConsumerProperties; } - - public int getDropPartitionsOlderThanHours() { - return dropPartitionsOlderThanHours; - } - - public int getCreatePartitionsInAdvanceHours() { - return createPartitionsInAdvanceHours; - } public String getKerberosHost() { return kerberosHost; } @@ -263,4 +173,7 @@ public void setMaximumFileSize(long maximumFileSize) { public int getNumOfConsumers() { return numOfConsumers; } + public long getPrune_offset() { + return prune_offset; + } } \ No newline at end of file diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSPrune.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSPrune.java index 541cc1b0..da27092b 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSPrune.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSPrune.java @@ -62,8 +62,8 @@ public HDFSPrune(Config config, String topicName) throws IOException { fs.mkdirs(newFolderPath); LOGGER.info("Path "+path+" created."); } - - cutoff_epoch = System.currentTimeMillis() - 172800000L; // TODO: cutoff offset is 172800000L only for testing, parametrize it using Config.java. + long pruneOffset = config.getPrune_offset(); + cutoff_epoch = System.currentTimeMillis() - pruneOffset; // pruneOffset is parametrized in Config.java. Default value is 2 days in milliseconds. } public void prune() throws IOException { diff --git a/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java b/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java index a4491929..d753c546 100644 --- a/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java +++ b/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java @@ -72,12 +72,15 @@ public void kafkaAndAvroFullTest() throws InterruptedException { kafkaController.run(); // The avro files should be committed to HDFS now. Check the committed files for any errors. // There should be 20 files, 10 partitions with each having 2 files assigned to them. - // TODO: hdfsReadCheck(); does not work properly if pruning is enabled. Add checks for pruning etc. - /*try { - hdfsReadCheck(); - } catch (IOException e) { - throw new RuntimeException(e); - }*/ + // hdfsReadCheck(); does not work properly if pruning is enabled and prune offset is set too low, which causes the records to be pruned from the database. + if (config.getPrune_offset() == 157784760000L) { + try { + hdfsReadCheck(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + } @Test @@ -108,7 +111,7 @@ public void hdfsPruneTest() throws IOException { LOGGER.info("Path "+path+" created."); } - // TODO: Use either HDFS-file modification timestamps or avro-mapred for pruning. + // Use either HDFS-file modification timestamps or avro-mapred for pruning. // The records are in this AVRO format: // {"timestamp": 1650872092240000, "message": "25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn audit says hi!]", "directory": "jla02logger", "stream": "test:jla02logger:0", "host": "jla-02.default", "input": "imrelp:cfe-06-0.cfe-06.default:", "partition": "8", "offset": 8, "origin": "jla-02.default"} @@ -116,26 +119,33 @@ public void hdfsPruneTest() throws IOException { // MapReduce functionalities of the Hadoop cluster: https://hadoop.apache.org/docs/stable/hadoop-mapreduce-client/hadoop-mapreduce-client-core/MapReduceTutorial.html // Avro side of documentations for MapReduce: https://avro.apache.org/docs/1.11.1/mapreduce-guide/ - - // Another method for pruning aside using avro-mapred is to use modification timestamp of the avro-file stored in HDFS: // fs.setTimes(new Path(path+"/"+0.8), Long.parseUnsignedLong("1675930598000"), -1); // where mtime is modification time and atime is access time. -1 as input parameter leaves the original atime/mtime value as is. FileStatus[] fileStatuses = fs.listStatus(new Path(newFolderPath + "/")); long count = Arrays.stream(fileStatuses).count(); if (count != 0) { + if (config.getPrune_offset() != 157784760000L) { + Assertions.fail("There are files available in the database when there should be none."); + } + boolean delete = false; for (FileStatus a : fileStatuses) { // If all the files have their modification timestamp altered to mirror the final record timestamp, it is possible to prune the database based on the timestamps of the fileStatuses object. long convert = TimeUnit.MILLISECONDS.convert(a.getModificationTime(), TimeUnit.MICROSECONDS); // MICROSECONDS ARE NOT SUPPORTED, convert the microsecond epoch to milliseconds. // Delete old files if (convert < 1708343921000L) { - boolean delete = fs.delete(a.getPath(), true); + delete = fs.delete(a.getPath(), true); Assertions.assertTrue(delete); LOGGER.info("Deleted file " + a.getPath()); } } + Assertions.assertTrue(delete); + LOGGER.info("All files were pruned properly."); }else { - LOGGER.info("No files available!"); + if (config.getPrune_offset() == 157784760000L) { + Assertions.fail("There were no files available in the database when there should be."); + } + LOGGER.info("No files available as they were pruned properly already!"); } fs.close(); @@ -236,6 +246,7 @@ record = reader.next(record); } else { Assertions.assertEquals("{\"timestamp\": 1650872092243000, \"message\": \"25.04.2022 07:34:52.243 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error metric says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 13, \"origin\": \"jla-02.default\"}", record.toString()); looper = 0; + LOGGER.info("Partition " + partitionCounter + " passed assertions."); partitionCounter++; } } diff --git a/src/test/java/com/teragrep/cfe_39/application.properties b/src/test/java/com/teragrep/cfe_39/application.properties index 80fe3bb8..186b9856 100644 --- a/src/test/java/com/teragrep/cfe_39/application.properties +++ b/src/test/java/com/teragrep/cfe_39/application.properties @@ -1,22 +1,11 @@ # Kafka security configuration file -java.security.auth.login.config=/home/p000048u/IdeaProjects/cfe_39/etc/config.jaas +java.security.auth.login.config=/example_path/cfe_39/etc/config.jaas # Logger settings -log4j2.configurationFile=/home/p000048u/IdeaProjects/cfe_39/etc/log4j2.properties -# Galera replication and fragment sizes/unit -db.replicationEnabled=true -db.streamUnits=bytes -db.streamSize=64000000 -# Override table and partition default location? -db.overrideTableLocation=false -db.tableLocation=/path/to/storage -# Partitions to drop that are older than this, must be >0 -db.dropPartitionsOlderThanHours=6 -# Partitions to create in advance, must be >0 -db.createPartitionsInAdvanceHours=168 +log4j2.configurationFile=/example_path/cfe_39/etc/log4j2.properties # What topics are searched from kafka, regex queueTopicPattern=^testConsumerTopic-*$ -# Number of consumers created to the consumer group -numOfConsumers=1 +# Number of consumers created to the consumer groups +numOfConsumers=2 # Kafka bootstrap servers - 127.0.0.1:9094,127.0.0.2:9094,127.0.0.3:9094 consumer.bootstrap.servers=test # Offset, should not be touched @@ -38,5 +27,9 @@ consumer.max.poll.interval.ms=300000 # For testing only, remove for prod. consumer.useMockKafkaConsumer=true # AVRO -queueDirectory=/home/p000048u/IdeaProjects/cfe_39/etc/AVRO/ -queueNamePrefix=testingAVRO \ No newline at end of file +queueDirectory=/example_path/cfe_39/etc/AVRO/ +queueNamePrefix=testingAVRO +# The maximum file size for AVRO-files that are to be stored in HDFS database. +maximumFileSize=3000 +# HDFS pruning, use 157784760000 value while testing HDFS writes to ensure the test records are not pruned. +prune_offset=157784760000 \ No newline at end of file From 0426a783324b3fc552055842389ab7e0dc0acf5f Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Wed, 21 Feb 2024 08:26:04 +0200 Subject: [PATCH 051/146] Added the missing kerberized access to HDFSPrune.java. --- .../cfe_39/consumers/kafka/HDFSPrune.java | 105 ++++++++++++++---- 1 file changed, 81 insertions(+), 24 deletions(-) diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSPrune.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSPrune.java index da27092b..b5f24bc0 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSPrune.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSPrune.java @@ -21,12 +21,16 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocalFileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.apache.hadoop.security.UserGroupInformation; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; import java.net.URI; +import java.util.Properties; import java.util.concurrent.TimeUnit; public class HDFSPrune { @@ -35,32 +39,85 @@ public class HDFSPrune { private final FileSystem fs; private Path newFolderPath; private long cutoff_epoch; + private final boolean useMockKafkaConsumer; // test-mode switch public HDFSPrune(Config config, String topicName) throws IOException { - this.config = config; - String hdfsuri = config.getHdfsuri(); - - String path = config.getHdfsPath()+"/"+topicName; - // ====== Init HDFS File System Object - Configuration conf = new Configuration(); - // Set FileSystem URI - conf.set("fs.defaultFS", hdfsuri); - // Because of Maven - conf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); - conf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); - // Set HADOOP user - System.setProperty("HADOOP_USER_NAME", "hdfs"); - System.setProperty("hadoop.home.dir", "/"); - //Get the filesystem - HDFS - fs = FileSystem.get(URI.create(hdfsuri), conf); - - //==== Create folder if not exists - Path workingDir=fs.getWorkingDirectory(); - newFolderPath= new Path(path); - if(!fs.exists(newFolderPath)) { - // Create new Directory - fs.mkdirs(newFolderPath); - LOGGER.info("Path "+path+" created."); + + // Check for testmode from config. + Properties readerKafkaProperties = config.getKafkaConsumerProperties(); + this.useMockKafkaConsumer = Boolean.parseBoolean( + readerKafkaProperties.getProperty("useMockKafkaConsumer", "false") + ); + + if (useMockKafkaConsumer) { + this.config = config; + String hdfsuri = config.getHdfsuri(); + + String path = config.getHdfsPath() + "/" + topicName; + // ====== Init HDFS File System Object + Configuration conf = new Configuration(); + // Set FileSystem URI + conf.set("fs.defaultFS", hdfsuri); + // Because of Maven + conf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); + conf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); + // Set HADOOP user + System.setProperty("HADOOP_USER_NAME", "hdfs"); + System.setProperty("hadoop.home.dir", "/"); + //Get the filesystem - HDFS + fs = FileSystem.get(URI.create(hdfsuri), conf); + + //==== Create folder if not exists + Path workingDir = fs.getWorkingDirectory(); + newFolderPath = new Path(path); + if (!fs.exists(newFolderPath)) { + // Create new Directory + fs.mkdirs(newFolderPath); + LOGGER.info("Path " + path + " created."); + } + }else { + // Code for initializing the class with kerberos. + String hdfsuri = config.getHdfsuri(); // Get from config. + + String path = config.getHdfsPath() + "/" + topicName; + + // set kerberos host and realm + System.setProperty("java.security.krb5.realm", config.getKerberosRealm()); + System.setProperty("java.security.krb5.kdc", config.getKerberosHost()); + + Configuration conf = new Configuration(); + + // enable kerberus + conf.set("hadoop.security.authentication", config.getHadoopAuthentication()); + conf.set("hadoop.security.authorization", config.getHadoopAuthorization()); + + conf.set("fs.defaultFS", hdfsuri); // Set FileSystem URI + conf.set("fs.hdfs.impl", DistributedFileSystem.class.getName()); // Maven stuff? + conf.set("fs.file.impl", LocalFileSystem.class.getName()); // Maven stuff? + + // hack for running locally with fake DNS records + // set this to true if overriding the host name in /etc/hosts + conf.set("dfs.client.use.datanode.hostname", config.getKerberosTestMode()); + + // server principal + // the kerberos principle that the namenode is using + conf.set("dfs.namenode.kerberos.principal.pattern", config.getKerberosPrincipal()); + + // set usergroup stuff + UserGroupInformation.setConfiguration(conf); + UserGroupInformation.loginUserFromKeytab(config.getKerberosKeytabUser(), config.getKerberosKeytabPath()); + + // filesystem for HDFS access is set here + fs = FileSystem.get(conf); + + //==== Create folder if not exists + Path workingDir = fs.getWorkingDirectory(); + newFolderPath = new Path(path); + if (!fs.exists(newFolderPath)) { + // Create new Directory + fs.mkdirs(newFolderPath); + LOGGER.info("Path " + path + " created."); + } } long pruneOffset = config.getPrune_offset(); cutoff_epoch = System.currentTimeMillis() - pruneOffset; // pruneOffset is parametrized in Config.java. Default value is 2 days in milliseconds. From 989b9e124c7c15984cc9b775640f23633a648857 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Wed, 21 Feb 2024 15:10:14 +0200 Subject: [PATCH 052/146] Implementing fixes/improvements from code review. Fixes for comments 1, 2, 3, 4, 5, 6, 7, 9, 11, 12 and 13. --- src/main/java/com/teragrep/cfe_39/Config.java | 16 +++++---- .../consumers/kafka/DatabaseOutput.java | 36 ++++++++++++------- .../cfe_39/consumers/kafka/HDFSPrune.java | 19 +++++----- .../cfe_39/consumers/kafka/HDFSWriter.java | 18 +++++----- .../com/teragrep/cfe_39/CombinedFullTest.java | 24 ++++++------- .../teragrep/cfe_39/application.properties | 2 +- 6 files changed, 66 insertions(+), 49 deletions(-) diff --git a/src/main/java/com/teragrep/cfe_39/Config.java b/src/main/java/com/teragrep/cfe_39/Config.java index a8692da9..6f664291 100644 --- a/src/main/java/com/teragrep/cfe_39/Config.java +++ b/src/main/java/com/teragrep/cfe_39/Config.java @@ -24,6 +24,7 @@ import java.io.File; import java.io.IOException; +import java.io.InputStream; import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; @@ -49,7 +50,7 @@ public class Config { private final String kerberosTestMode; private long maximumFileSize; private final int numOfConsumers; - private final long prune_offset; + private final long pruneOffset; // TODO: Set up configuration check for important parameters. Remove old unused parameters. @@ -57,15 +58,18 @@ public class Config { Properties properties = new Properties(); Path configPath = Paths.get(System.getProperty("cfe_30.config.location", System.getProperty("user.dir") + "/etc/application.properties")); LOGGER.info("Loading application config '" + configPath.toAbsolutePath() + "'"); - properties.load(Files.newInputStream(configPath)); - LOGGER.debug("Got configuration: " + properties); + + try(InputStream inputStream = Files.newInputStream(configPath)) { + properties.load(inputStream); + LOGGER.debug("Got configuration: " + properties); + } // HDFS this.hdfsPath = properties.getProperty("hdfsPath", "hdfs:///opt/teragrep/cfe_39/srv/"); this.hdfsuri = properties.getProperty("hdfsuri", "hdfs://localhost:45937/"); // HDFS pruning - this.prune_offset = Long.parseLong(properties.getProperty("prune_offset", "172800000")); + this.pruneOffset = Long.parseLong(properties.getProperty("pruneOffset", "172800000")); // AVRO this.queueDirectory = properties.getProperty("queueDirectory", ""); @@ -173,7 +177,7 @@ public void setMaximumFileSize(long maximumFileSize) { public int getNumOfConsumers() { return numOfConsumers; } - public long getPrune_offset() { - return prune_offset; + public long getPruneOffset() { + return pruneOffset; } } \ No newline at end of file diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java index a4415342..6aba87a0 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java @@ -56,7 +56,7 @@ public class DatabaseOutput implements Consumer> { public static final String ANSI_GREEN = "\u001B[32m"; public static final String ANSI_BLUE = "\u001B[34m"; private SyslogAvroWriter syslogAvroWriter; - private final long minimumFreeSpace; +// private final long minimumFreeSpace; // TODO: Check if needs to be implemented to the code. private final long maximumFileSize; private final WritableQueue writableQueue; private final ByteBuffer sourceConcatenationBuffer; @@ -71,7 +71,7 @@ public class DatabaseOutput implements Consumer> { private final SDVector originHostname; private File syslogFile; private final Config config; - private long approximatedSize; // FIXME: Not working properly when flush() is not used after append in the AVRO-file. +// private long approximatedSize; // FIXME: Not working properly when flush() is not used after append in the AVRO-file. private long epochMicros_last; DatabaseOutput( @@ -84,7 +84,7 @@ public class DatabaseOutput implements Consumer> { this.table = table; this.durationStatistics = durationStatistics; this.topicCounter = topicCounter; - this.minimumFreeSpace = 32000000; // TODO: CHECK RIGHT VALUE FOR minimumFreeSpace +// this.minimumFreeSpace = 32000000; // TODO: CHECK RIGHT VALUE FOR minimumFreeSpace this.maximumFileSize = config.getMaximumFileSize();; // Maximum file size should be 64M (64000000). 60800000 is 95% of 64M which should be a good approximation point. // queueDirectory and queueNamePrefix are only used for temporarily storing the AVRO-serialized files before committing them to HDFS when the file size reaches the threshold (or all records are processed). @@ -102,12 +102,12 @@ public class DatabaseOutput implements Consumer> { this.eventNodeSourceHostname = new SDVector("event_node_source@48577","hostname"); this.eventNodeRelayHostname = new SDVector("event_node_relay@48577","hostname"); this.originHostname = new SDVector("origin@48577","hostname"); - this.approximatedSize = 0; +// this.approximatedSize = 0; this.epochMicros_last = 0L; } - // Checks that the filesize stays under the defined maximum file size. - boolean checkSizeTooLarge(long fileSize, RecordOffsetObject recordOffsetObject) { + // Checks that the filesize stays under the defined maximum file size. If the file is about to go over target limit commits the file to HDFS and returns true, otherwise does nothing and returns false. + boolean committedToHdfs(long fileSize, RecordOffsetObject recordOffsetObject) { try { // If the syslogAvroWriter is already initialized, check the filesize so it doesn't go above maximumFileSize. if (fileSize > maximumFileSize) { @@ -123,7 +123,7 @@ boolean checkSizeTooLarge(long fileSize, RecordOffsetObject recordOffsetObject) writableQueue.setQueueNamePrefix(recordOffsetObject.topic+recordOffsetObject.partition); syslogFile = writableQueue.getNextWritableFile(); syslogAvroWriter = new SyslogAvroWriter(syslogFile); - approximatedSize = syslogAvroWriter.getFileSize(); // resets the size approximation. +// approximatedSize = syslogAvroWriter.getFileSize(); // resets the size approximation. return true; } } catch (IOException ioException) { @@ -172,7 +172,7 @@ public void accept(List recordOffsetObjectList) { writableQueue.getNextWritableFile(); // The HDFS filename is only finalized when the AVRO-serialized file is finalized, because every Kafka-record added to the file is going to change the offset that is going to be used for the filename. syslogAvroWriter = new SyslogAvroWriter(syslogFile); - approximatedSize = syslogAvroWriter.getFileSize(); // resets the size approximation. +// approximatedSize = syslogAvroWriter.getFileSize(); // resets the size approximation. lastObject = recordOffsetObject; } catch (IOException ioException) { throw new IllegalArgumentException(ioException); @@ -183,7 +183,12 @@ public void accept(List recordOffsetObjectList) { assert lastObject != null; if (lastObject.topic.equals(recordOffsetObject.topic) & lastObject.partition.equals(recordOffsetObject.partition)) { // Records left to consume in the current partition. - checkSizeTooLarge(syslogAvroWriter.getFileSize(), lastObject); + boolean b = committedToHdfs(syslogAvroWriter.getFileSize(), lastObject); + if (b) { + LOGGER.debug("Target file size reached, file {} stored to {} in HDFS", syslogFile.getName(), lastObject.topic+"/"+lastObject.partition+"."+lastObject.offset); + }else { + LOGGER.debug("Target file size not yet reached, continuing writing records to {}.", syslogFile.getName()); + } } else { // Previous partition was fully consumed. Commit file to HDFS and create a new AVRO-file. syslogAvroWriter.close(); @@ -195,7 +200,7 @@ public void accept(List recordOffsetObjectList) { writableQueue.setQueueNamePrefix(recordOffsetObject.topic+recordOffsetObject.partition); syslogFile = writableQueue.getNextWritableFile(); syslogAvroWriter = new SyslogAvroWriter(syslogFile); - approximatedSize = syslogAvroWriter.getFileSize(); // resets the size approximation. +// approximatedSize = syslogAvroWriter.getFileSize(); // resets the size approximation. } } catch (IOException ioException) { throw new UncheckedIOException(ioException); @@ -256,13 +261,18 @@ public void accept(List recordOffsetObjectList) { epochMicros_last = epochMicros; } // Check if there is still room in syslogAvroWriter for another syslogRecord. Commit syslogAvroWriter to HDFS if no room left, emptying it out in the process. - // checkSizeTooLarge(approximatedSize + capacity, lastObject); // FIXME: approximatedSize is not working properly without the use of flush() after append. File sizes are all over the place. - checkSizeTooLarge(syslogAvroWriter.getFileSize() + capacity, lastObject); + // boolean b = committedToHdfs(approximatedSize + capacity, lastObject); // FIXME: approximatedSize is not working properly without the use of flush() after append. File sizes are all over the place. + boolean b = committedToHdfs(syslogAvroWriter.getFileSize() + capacity, lastObject); + if (b) { + LOGGER.debug("Target file size reached, file {} stored to {} in HDFS", syslogFile.getName(), lastObject.topic+"/"+lastObject.partition+"."+lastObject.offset); + }else { + LOGGER.debug("Target file size not yet reached, continuing writing records to {}.", syslogFile.getName()); + } // if more records can be inserted, update epochMicros_last with the timestamp of the last inserted record. epochMicros_last = epochMicros; // Add syslogRecord to syslogAvroWriter which has room for new syslogRecord. syslogAvroWriter.write(syslogRecord); - approximatedSize += capacity; +// approximatedSize += capacity; lastObject = recordOffsetObject; // The difference between actual and approximate file size is about 2,4 % with 64M files. So setting the MaximumFileSize to 95 % of the target should make things work. } diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSPrune.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSPrune.java index b5f24bc0..9aee5766 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSPrune.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSPrune.java @@ -38,7 +38,7 @@ public class HDFSPrune { private Config config; private final FileSystem fs; private Path newFolderPath; - private long cutoff_epoch; + private long cutOffEpoch; private final boolean useMockKafkaConsumer; // test-mode switch public HDFSPrune(Config config, String topicName) throws IOException { @@ -52,8 +52,7 @@ public HDFSPrune(Config config, String topicName) throws IOException { if (useMockKafkaConsumer) { this.config = config; String hdfsuri = config.getHdfsuri(); - - String path = config.getHdfsPath() + "/" + topicName; + String path = config.getHdfsPath().concat("/").concat(topicName); // ====== Init HDFS File System Object Configuration conf = new Configuration(); // Set FileSystem URI @@ -73,7 +72,7 @@ public HDFSPrune(Config config, String topicName) throws IOException { if (!fs.exists(newFolderPath)) { // Create new Directory fs.mkdirs(newFolderPath); - LOGGER.info("Path " + path + " created."); + LOGGER.info("Path {} created.", path); } }else { // Code for initializing the class with kerberos. @@ -116,24 +115,28 @@ public HDFSPrune(Config config, String topicName) throws IOException { if (!fs.exists(newFolderPath)) { // Create new Directory fs.mkdirs(newFolderPath); - LOGGER.info("Path " + path + " created."); + LOGGER.info("Path {} created.", path); } } - long pruneOffset = config.getPrune_offset(); - cutoff_epoch = System.currentTimeMillis() - pruneOffset; // pruneOffset is parametrized in Config.java. Default value is 2 days in milliseconds. + long pruneOffset = config.getPruneOffset(); + cutOffEpoch = System.currentTimeMillis() - pruneOffset; // pruneOffset is parametrized in Config.java. Default value is 2 days in milliseconds. } public void prune() throws IOException { // Fetch the filestatuses of HDFS files. FileStatus[] fileStatuses = fs.listStatus(new Path(newFolderPath + "/")); + if (fileStatuses.length > 0) { for (FileStatus a : fileStatuses) { // If all the files have their modification timestamp altered to mirror the final record timestamp, it is possible to prune the database based on the timestamps of the fileStatuses object. long convert = TimeUnit.MILLISECONDS.convert(a.getModificationTime(), TimeUnit.MICROSECONDS); // MICROSECONDS ARE NOT SUPPORTED, convert the microsecond epoch to milliseconds. // Delete old files - if (convert < cutoff_epoch) { + if (convert < cutOffEpoch) { boolean delete = fs.delete(a.getPath(), true); LOGGER.info("Deleted file " + a.getPath()); } } + } else { + LOGGER.info("No files found in folder {}", new Path(newFolderPath + "/")); + } } } diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java index 815c8675..379a5680 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java @@ -128,7 +128,7 @@ public HDFSWriter(Config config, RecordOffsetObject lastObject) throws IOExcepti } // Method for committing the AVRO-file to HDFS - public void commit(File syslogFile, long epochMicros_last) { + public void commit(File syslogFile, long lastEpochMicros) { // The code for writing the file to HDFS should be same for both test (non-kerberized access) and prod (kerberized access). if (useMockKafkaConsumer) { // CODE FOR TEST-MODE GOES HERE! @@ -141,7 +141,7 @@ public void commit(File syslogFile, long epochMicros_last) { if (!fs.exists(newFolderPath)) { // Create new Directory fs.mkdirs(newFolderPath); - LOGGER.debug("Path "+path+" created."); + LOGGER.debug("Path {} created.", path); } //==== Write file @@ -167,11 +167,11 @@ public void commit(File syslogFile, long epochMicros_last) { Path path = new Path(syslogFile.getPath()); fs.copyFromLocalFile(path, hdfswritepath); - fs.setTimes(hdfswritepath, epochMicros_last, -1); // where mtime is modification time and atime is access time. -1 as input parameter leaves the original atime/mtime value as is. - // updateTimestamp(hdfswritepath, epochMicros_last); + fs.setTimes(hdfswritepath, lastEpochMicros, -1); // where mtime is modification time and atime is access time. -1 as input parameter leaves the original atime/mtime value as is. + // updateTimestamp(hdfswritepath, lastEpochMicros); LOGGER.debug("End Write file into hdfs"); boolean delete = syslogFile.delete(); // deletes the avro-file from the local disk now that it has been committed to HDFS. - LOGGER.debug("\n" + "File committed to HDFS, file writepath should be: " + hdfswritepath.toString() + "\n"); + LOGGER.debug("\nFile committed to HDFS, file writepath should be: {}\n", hdfswritepath.toString()); } catch (IOException e) { throw new RuntimeException(e); @@ -186,7 +186,7 @@ public void commit(File syslogFile, long epochMicros_last) { if (!fs.exists(newFolderPath)) { // Create new Directory fs.mkdirs(newFolderPath); - LOGGER.debug("Path "+path+" created."); + LOGGER.debug("Path {} created.", path); } //==== Write file @@ -212,7 +212,7 @@ public void commit(File syslogFile, long epochMicros_last) { outputStream.close(); LOGGER.debug("End Write file into hdfs"); boolean delete = syslogFile.delete(); // deletes the avro-file from the local disk now that it has been committed to HDFS. - LOGGER.debug("\n" + "File committed to HDFS, file writepath should be: " + hdfswritepath.toString() + "\n"); + LOGGER.debug("\nFile committed to HDFS, file writepath should be: {}\n", hdfswritepath.toString()); } catch (IOException e) { throw new RuntimeException(e); @@ -220,12 +220,12 @@ public void commit(File syslogFile, long epochMicros_last) { } } - private void updateTimestamp(Path hdfswritepath, long epochMicros_last) { + private void updateTimestamp(Path hdfswritepath, long lastEpochMicros) { // Testing timestamp editing. The new timestamp should be the timestamp of the last record that was added to the AVRO-file. try { FileSystem fs_temp = FileSystem.get(URI.create(hdfsuri), conf); FSDataOutputStream fsDataOutputStream = fs_temp.create(hdfswritepath); - fs_temp.setTimes(hdfswritepath, epochMicros_last, -1); // where mtime is modification time and atime is access time. -1 as input parameter leaves the original atime/mtime value as is. + fs_temp.setTimes(hdfswritepath, lastEpochMicros, -1); // where mtime is modification time and atime is access time. -1 as input parameter leaves the original atime/mtime value as is. fsDataOutputStream.close(); } catch (IOException e) { throw new RuntimeException(e); diff --git a/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java b/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java index d753c546..bdb9af9d 100644 --- a/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java +++ b/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java @@ -39,11 +39,11 @@ public static void startMiniCluster() throws IOException, InterruptedException { try { config = new Config(); } catch (IOException e){ - LOGGER.error("Can't load config: " + e); - System.exit(1); + LOGGER.error("Can't load config: {}", e.toString()); + Assertions.fail(); } catch (IllegalArgumentException e) { - LOGGER.error("Got invalid config: " + e); - System.exit(1); + LOGGER.error("Got invalid config: {}", e.toString()); + Assertions.fail(); } // Create a HDFS miniCluster baseDir = Files.createTempDirectory("test_hdfs").toFile().getAbsoluteFile(); @@ -73,7 +73,7 @@ public void kafkaAndAvroFullTest() throws InterruptedException { // The avro files should be committed to HDFS now. Check the committed files for any errors. // There should be 20 files, 10 partitions with each having 2 files assigned to them. // hdfsReadCheck(); does not work properly if pruning is enabled and prune offset is set too low, which causes the records to be pruned from the database. - if (config.getPrune_offset() == 157784760000L) { + if (config.getPruneOffset() == 157784760000L) { try { hdfsReadCheck(); } catch (IOException e) { @@ -108,7 +108,7 @@ public void hdfsPruneTest() throws IOException { if(!fs.exists(newFolderPath)) { // Create new Directory fs.mkdirs(newFolderPath); - LOGGER.info("Path "+path+" created."); + LOGGER.info("Path {} created.", path); } // Use either HDFS-file modification timestamps or avro-mapred for pruning. @@ -125,7 +125,7 @@ public void hdfsPruneTest() throws IOException { FileStatus[] fileStatuses = fs.listStatus(new Path(newFolderPath + "/")); long count = Arrays.stream(fileStatuses).count(); if (count != 0) { - if (config.getPrune_offset() != 157784760000L) { + if (config.getPruneOffset() != 157784760000L) { Assertions.fail("There are files available in the database when there should be none."); } boolean delete = false; @@ -136,13 +136,13 @@ public void hdfsPruneTest() throws IOException { if (convert < 1708343921000L) { delete = fs.delete(a.getPath(), true); Assertions.assertTrue(delete); - LOGGER.info("Deleted file " + a.getPath()); + LOGGER.info("Deleted file {}", a.getPath()); } } Assertions.assertTrue(delete); LOGGER.info("All files were pruned properly."); }else { - if (config.getPrune_offset() == 157784760000L) { + if (config.getPruneOffset() == 157784760000L) { Assertions.fail("There were no files available in the database when there should be."); } LOGGER.info("No files available as they were pruned properly already!"); @@ -175,7 +175,7 @@ public void hdfsReadCheck() throws IOException { if(!fs.exists(newFolderPath)) { // Create new Directory fs.mkdirs(newFolderPath); - LOGGER.info("Path "+path+" created."); + LOGGER.info("Path {} created.", path); } // This is the HDFS write path for the files: @@ -199,7 +199,7 @@ public void hdfsReadCheck() throws IOException { //The data is in AVRO-format, so it can't be read as a string. DataFileStream reader = new DataFileStream<>(inputStream, new SpecificDatumReader<>(SyslogRecord.class)); SyslogRecord record = null; - LOGGER.info("\nReading records from file " + hdfsreadpath.toString() + ":"); + LOGGER.info("\nReading records from file {}:", hdfsreadpath.toString()); while (reader.hasNext()) { record = reader.next(record); LOGGER.info(record.toString()); @@ -246,7 +246,7 @@ record = reader.next(record); } else { Assertions.assertEquals("{\"timestamp\": 1650872092243000, \"message\": \"25.04.2022 07:34:52.243 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error metric says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 13, \"origin\": \"jla-02.default\"}", record.toString()); looper = 0; - LOGGER.info("Partition " + partitionCounter + " passed assertions."); + LOGGER.info("Partition {} passed assertions.", partitionCounter); partitionCounter++; } } diff --git a/src/test/java/com/teragrep/cfe_39/application.properties b/src/test/java/com/teragrep/cfe_39/application.properties index 186b9856..4cc21a53 100644 --- a/src/test/java/com/teragrep/cfe_39/application.properties +++ b/src/test/java/com/teragrep/cfe_39/application.properties @@ -32,4 +32,4 @@ queueNamePrefix=testingAVRO # The maximum file size for AVRO-files that are to be stored in HDFS database. maximumFileSize=3000 # HDFS pruning, use 157784760000 value while testing HDFS writes to ensure the test records are not pruned. -prune_offset=157784760000 \ No newline at end of file +pruneOffset=157784760000 \ No newline at end of file From fb67b46f857927f6b9e70224da0a7993b568037f Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Mon, 26 Feb 2024 16:19:40 +0200 Subject: [PATCH 053/146] Implemented null object pattern for RecordOffsetObject (improvements from code review comments 14 and 15). --- .../consumers/kafka/DatabaseOutput.java | 49 +++++++++---------- .../consumers/kafka/NullOffsetObject.java | 45 +++++++++++++++++ .../cfe_39/consumers/kafka/OffsetObject.java | 29 +++++++++++ .../consumers/kafka/RecordOffsetObject.java | 31 ++++++++++-- .../java/com/teragrep/cfe_39/HdfsTest.java | 2 +- 5 files changed, 124 insertions(+), 32 deletions(-) create mode 100644 src/main/java/com/teragrep/cfe_39/consumers/kafka/NullOffsetObject.java create mode 100644 src/main/java/com/teragrep/cfe_39/consumers/kafka/OffsetObject.java diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java index 6aba87a0..0ecd965b 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java @@ -56,7 +56,7 @@ public class DatabaseOutput implements Consumer> { public static final String ANSI_GREEN = "\u001B[32m"; public static final String ANSI_BLUE = "\u001B[34m"; private SyslogAvroWriter syslogAvroWriter; -// private final long minimumFreeSpace; // TODO: Check if needs to be implemented to the code. + // private final long minimumFreeSpace; // TODO: Check if needs to be implemented to the code. private final long maximumFileSize; private final WritableQueue writableQueue; private final ByteBuffer sourceConcatenationBuffer; @@ -71,7 +71,7 @@ public class DatabaseOutput implements Consumer> { private final SDVector originHostname; private File syslogFile; private final Config config; -// private long approximatedSize; // FIXME: Not working properly when flush() is not used after append in the AVRO-file. + // private long approximatedSize; // FIXME: Not working properly when flush() is not used after append in the AVRO-file. private long epochMicros_last; DatabaseOutput( @@ -85,7 +85,7 @@ public class DatabaseOutput implements Consumer> { this.durationStatistics = durationStatistics; this.topicCounter = topicCounter; // this.minimumFreeSpace = 32000000; // TODO: CHECK RIGHT VALUE FOR minimumFreeSpace - this.maximumFileSize = config.getMaximumFileSize();; // Maximum file size should be 64M (64000000). 60800000 is 95% of 64M which should be a good approximation point. + this.maximumFileSize = config.getMaximumFileSize(); // Maximum file size should be 64M (64000000). 60800000 is 95% of 64M which should be a good approximation point. // queueDirectory and queueNamePrefix are only used for temporarily storing the AVRO-serialized files before committing them to HDFS when the file size reaches the threshold (or all records are processed). this.writableQueue = new WritableQueue( @@ -120,7 +120,7 @@ boolean committedToHdfs(long fileSize, RecordOffsetObject recordOffsetObject) { } // This part defines a new empty file to which the new AVRO-serialized records are stored until it again hits the 64M size limit. - writableQueue.setQueueNamePrefix(recordOffsetObject.topic+recordOffsetObject.partition); + writableQueue.setQueueNamePrefix(recordOffsetObject.getTopic()+recordOffsetObject.getPartition()); syslogFile = writableQueue.getNextWritableFile(); syslogAvroWriter = new SyslogAvroWriter(syslogFile); // approximatedSize = syslogAvroWriter.getFileSize(); // resets the size approximation. @@ -160,14 +160,14 @@ public void accept(List recordOffsetObjectList) { // When the file size is about to go above 64M, commit the file into HDFS using the latest topic/partition/offset values as the filename and start fresh with a new empty AVRO-file. // Serialize the object that was going to make the file go above 64M into the now empty AVRO-file and continue the loop. // TODO: If the prod-environment recordOffsetObjectList ordering is different from what it is in the test environment, add a function that reorders the list based on partition and offset (or better yet, make several AVRO-files that are being used at the same time rather than doing it one AVRO-file at a time as the offset ordering within partitions should always be correct in all scenarios). - RecordOffsetObject lastObject = null; + OffsetObject lastObject = new NullOffsetObject(); // Abstract OffsetObject class, set to null object before initializing as RecordOffsetObject. long start = Instant.now().toEpochMilli(); // Starts measuring performance here. Measures how long it takes to process the whole recordOffsetObjectList. // This loop goes through all the records of the mock data in a single session. for (RecordOffsetObject recordOffsetObject : recordOffsetObjectList) { - // Initializing syslogAvroWriter. - if (syslogAvroWriter == null) { + // Initializing syslogAvroWriter and lastObject. + if (syslogAvroWriter == null && lastObject.isNull()) { try { - writableQueue.setQueueNamePrefix(recordOffsetObject.topic+recordOffsetObject.partition); + writableQueue.setQueueNamePrefix(recordOffsetObject.getTopic()+recordOffsetObject.getPartition()); syslogFile = writableQueue.getNextWritableFile(); // The HDFS filename is only finalized when the AVRO-serialized file is finalized, because every Kafka-record added to the file is going to change the offset that is going to be used for the filename. @@ -179,35 +179,32 @@ public void accept(List recordOffsetObjectList) { } } else { try { - // check if new partition - assert lastObject != null; - if (lastObject.topic.equals(recordOffsetObject.topic) & lastObject.partition.equals(recordOffsetObject.partition)) { + if (lastObject.getTopic().equals(recordOffsetObject.getTopic()) & lastObject.getPartition().equals(recordOffsetObject.getPartition())) { // Records left to consume in the current partition. - boolean b = committedToHdfs(syslogAvroWriter.getFileSize(), lastObject); + boolean b = committedToHdfs(syslogAvroWriter.getFileSize(), (RecordOffsetObject) lastObject); if (b) { - LOGGER.debug("Target file size reached, file {} stored to {} in HDFS", syslogFile.getName(), lastObject.topic+"/"+lastObject.partition+"."+lastObject.offset); - }else { + LOGGER.debug("Target file size reached, file {} stored to {} in HDFS", syslogFile.getName(), lastObject.getTopic() + "/" + lastObject.getPartition() + "." + lastObject.getOffset()); + } else { LOGGER.debug("Target file size not yet reached, continuing writing records to {}.", syslogFile.getName()); } } else { // Previous partition was fully consumed. Commit file to HDFS and create a new AVRO-file. syslogAvroWriter.close(); - try (HDFSWriter writer = new HDFSWriter(config, lastObject)) { - writer.commit(syslogFile, epochMicros_last); - } + HDFSWriter writer = new HDFSWriter(config, (RecordOffsetObject) lastObject); + writer.commit(syslogFile, epochMicros_last); // This part defines a new empty file to which the new AVRO-serialized records are stored until it again hits the 64M size limit. - writableQueue.setQueueNamePrefix(recordOffsetObject.topic+recordOffsetObject.partition); + writableQueue.setQueueNamePrefix(recordOffsetObject.getTopic() + recordOffsetObject.getPartition()); syslogFile = writableQueue.getNextWritableFile(); syslogAvroWriter = new SyslogAvroWriter(syslogFile); -// approximatedSize = syslogAvroWriter.getFileSize(); // resets the size approximation. +// approximatedSize = syslogAvroWriter.getFileSize(); // resets the size approximation. } } catch (IOException ioException) { throw new UncheckedIOException(ioException); } } - byte[] byteArray = recordOffsetObject.record; // loads the byte[] contained in recordOffsetObject.record to byteArray. + byte[] byteArray = recordOffsetObject.getRecord(); // loads the byte[] contained in recordOffsetObject.getRecord() to byteArray. batchBytes = batchBytes + byteArray.length; InputStream inputStream = new ByteArrayInputStream(byteArray); rfc5424Frame.load(inputStream); @@ -248,8 +245,8 @@ public void accept(List recordOffsetObjectList) { .setStream(rfc5424Frame.structuredData.getValue(teragrepStreamName).toString()) // Or is sourcetype/stream supposed to be rfc5424Frame.appName.toString() instead? .setHost(rfc5424Frame.hostname.toString()) .setInput(new String(source, StandardCharsets.UTF_8)) - .setPartition(recordOffsetObject.partition.toString()) - .setOffset(recordOffsetObject.offset) + .setPartition(recordOffsetObject.getPartition().toString()) + .setOffset(recordOffsetObject.getOffset()) .setOrigin(new String(origin, StandardCharsets.UTF_8)) .build(); @@ -262,9 +259,9 @@ public void accept(List recordOffsetObjectList) { } // Check if there is still room in syslogAvroWriter for another syslogRecord. Commit syslogAvroWriter to HDFS if no room left, emptying it out in the process. // boolean b = committedToHdfs(approximatedSize + capacity, lastObject); // FIXME: approximatedSize is not working properly without the use of flush() after append. File sizes are all over the place. - boolean b = committedToHdfs(syslogAvroWriter.getFileSize() + capacity, lastObject); + boolean b = committedToHdfs(syslogAvroWriter.getFileSize() + capacity, (RecordOffsetObject) lastObject); if (b) { - LOGGER.debug("Target file size reached, file {} stored to {} in HDFS", syslogFile.getName(), lastObject.topic+"/"+lastObject.partition+"."+lastObject.offset); + LOGGER.debug("Target file size reached, file {} stored to {} in HDFS", syslogFile.getName(), lastObject.getTopic()+"/"+lastObject.getPartition()+"."+lastObject.getOffset()); }else { LOGGER.debug("Target file size not yet reached, continuing writing records to {}.", syslogFile.getName()); } @@ -283,9 +280,9 @@ public void accept(List recordOffsetObjectList) { // Handle the "leftover" syslogRecords from the loop. try { - if (syslogAvroWriter != null) { + if (syslogAvroWriter != null && !lastObject.isNull()) { syslogAvroWriter.close(); - try (HDFSWriter writer = new HDFSWriter(config, lastObject)) { + try (HDFSWriter writer = new HDFSWriter(config, (RecordOffsetObject) lastObject)) { writer.commit(syslogFile, epochMicros_last); // commits the final AVRO-file to HDFS. } } diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/NullOffsetObject.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/NullOffsetObject.java new file mode 100644 index 00000000..fe7166af --- /dev/null +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/NullOffsetObject.java @@ -0,0 +1,45 @@ +/* + HDFS Data Ingestion for PTH_06 use CFE-39 + Copyright (C) 2022 Fail-Safe IT Solutions Oy + + 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. + */ +package com.teragrep.cfe_39.consumers.kafka; + +public class NullOffsetObject extends OffsetObject{ + + @Override + public boolean isNull() { + return true; + } + + @Override + public String getTopic() { + return "Not available"; + } + + @Override + public Integer getPartition() { + return 0; + } + + @Override + public Long getOffset() { + return 0L; + } + + @Override + public byte[] getRecord() { + return new byte[0]; + } +} diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/OffsetObject.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/OffsetObject.java new file mode 100644 index 00000000..cd123e0a --- /dev/null +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/OffsetObject.java @@ -0,0 +1,29 @@ +/* + HDFS Data Ingestion for PTH_06 use CFE-39 + Copyright (C) 2022 Fail-Safe IT Solutions Oy + + 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. + */ +package com.teragrep.cfe_39.consumers.kafka; + +public abstract class OffsetObject { + protected String topic; + protected Integer partition; + protected Long offset; + protected byte[] record; + public abstract boolean isNull(); + public abstract String getTopic(); + public abstract Integer getPartition(); + public abstract Long getOffset(); + public abstract byte[] getRecord(); +} diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/RecordOffsetObject.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/RecordOffsetObject.java index b0516965..69b736f6 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/RecordOffsetObject.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/RecordOffsetObject.java @@ -18,11 +18,7 @@ package com.teragrep.cfe_39.consumers.kafka; // This is the class for handling the Kafka record topic/partition/offset data that are required for HDFS storage. -public class RecordOffsetObject { - public String topic; - public Integer partition; - public Long offset; - public byte[] record; +public class RecordOffsetObject extends OffsetObject{ public RecordOffsetObject( String topic, @@ -35,4 +31,29 @@ public RecordOffsetObject( this.offset = offset; this.record = record; } + + @Override + public boolean isNull() { + return false; + } + + @Override + public String getTopic() { + return topic; + } + + @Override + public Integer getPartition() { + return partition; + } + + @Override + public Long getOffset() { + return offset; + } + + @Override + public byte[] getRecord() { + return record; + } } diff --git a/src/test/java/com/teragrep/cfe_39/HdfsTest.java b/src/test/java/com/teragrep/cfe_39/HdfsTest.java index bfbc831d..4f660d1e 100644 --- a/src/test/java/com/teragrep/cfe_39/HdfsTest.java +++ b/src/test/java/com/teragrep/cfe_39/HdfsTest.java @@ -137,7 +137,7 @@ public void hdfsWriteTest() { assert lastRecord != null; RecordOffsetObject lastObject = new RecordOffsetObject("testConsumerTopic", Integer.parseInt(lastRecord.getPartition().toString()), lastRecord.getOffset(), null); // Fetch input parameters from the lastRecord SyslogRecord-object. - LOGGER.debug("\n"+"Last record in the " + syslogFile.getName() + " file:" + "\ntopic: " + lastObject.topic + "\npartition: " + lastObject.partition + "\noffset: " + lastObject.offset); + LOGGER.debug("\n"+"Last record in the " + syslogFile.getName() + " file:" + "\ntopic: " + lastObject.getTopic() + "\npartition: " + lastObject.getPartition() + "\noffset: " + lastObject.getOffset()); try (HDFSWriter writer = new HDFSWriter(config, lastObject)) { writer.commit(syslogFile, -1L); // commits the final AVRO-file to HDFS. } catch (IOException e) { From 17dc70979341fc8e0e62b2f97c30587cb5011b44 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Wed, 13 Mar 2024 10:10:58 +0200 Subject: [PATCH 054/146] Reverted back to using original modification timestamp functionality of the FileSystem in HDFS and file pruning because Kafka records are not guaranteed to be ordered by time. --- .../java/com/teragrep/cfe_39/consumers/kafka/HDFSPrune.java | 4 +--- .../java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java | 2 +- src/test/java/com/teragrep/cfe_39/CombinedFullTest.java | 4 +--- 3 files changed, 3 insertions(+), 7 deletions(-) diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSPrune.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSPrune.java index 9aee5766..2268183a 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSPrune.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSPrune.java @@ -127,10 +127,8 @@ public void prune() throws IOException { FileStatus[] fileStatuses = fs.listStatus(new Path(newFolderPath + "/")); if (fileStatuses.length > 0) { for (FileStatus a : fileStatuses) { - // If all the files have their modification timestamp altered to mirror the final record timestamp, it is possible to prune the database based on the timestamps of the fileStatuses object. - long convert = TimeUnit.MILLISECONDS.convert(a.getModificationTime(), TimeUnit.MICROSECONDS); // MICROSECONDS ARE NOT SUPPORTED, convert the microsecond epoch to milliseconds. // Delete old files - if (convert < cutOffEpoch) { + if (a.getModificationTime() < cutOffEpoch) { boolean delete = fs.delete(a.getPath(), true); LOGGER.info("Deleted file " + a.getPath()); } diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java index 379a5680..76013528 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java @@ -167,7 +167,7 @@ public void commit(File syslogFile, long lastEpochMicros) { Path path = new Path(syslogFile.getPath()); fs.copyFromLocalFile(path, hdfswritepath); - fs.setTimes(hdfswritepath, lastEpochMicros, -1); // where mtime is modification time and atime is access time. -1 as input parameter leaves the original atime/mtime value as is. + // fs.setTimes(hdfswritepath, lastEpochMicros, -1); // where mtime is modification time and atime is access time. -1 as input parameter leaves the original atime/mtime value as is. // updateTimestamp(hdfswritepath, lastEpochMicros); LOGGER.debug("End Write file into hdfs"); boolean delete = syslogFile.delete(); // deletes the avro-file from the local disk now that it has been committed to HDFS. diff --git a/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java b/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java index bdb9af9d..92966d0d 100644 --- a/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java +++ b/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java @@ -130,10 +130,8 @@ public void hdfsPruneTest() throws IOException { } boolean delete = false; for (FileStatus a : fileStatuses) { - // If all the files have their modification timestamp altered to mirror the final record timestamp, it is possible to prune the database based on the timestamps of the fileStatuses object. - long convert = TimeUnit.MILLISECONDS.convert(a.getModificationTime(), TimeUnit.MICROSECONDS); // MICROSECONDS ARE NOT SUPPORTED, convert the microsecond epoch to milliseconds. // Delete old files - if (convert < 1708343921000L) { + if (a.getModificationTime() < System.currentTimeMillis()) { delete = fs.delete(a.getPath(), true); Assertions.assertTrue(delete); LOGGER.info("Deleted file {}", a.getPath()); From ffe194240791be6347089dfe6414fc7cf8887fa2 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Thu, 4 Apr 2024 14:44:39 +0300 Subject: [PATCH 055/146] Changed Offset class namings. Changed file system asset naming. --- .../consumers/kafka/DatabaseOutput.java | 18 ++++++------ .../cfe_39/consumers/kafka/HDFSPrune.java | 22 +++++++-------- .../cfe_39/consumers/kafka/HDFSWriter.java | 28 +++++++++---------- .../consumers/kafka/KafkaController.java | 2 +- .../cfe_39/consumers/kafka/KafkaReader.java | 8 +++--- ...{NullOffsetObject.java => NullOffset.java} | 2 +- .../kafka/{OffsetObject.java => Offset.java} | 2 +- .../consumers/kafka/ReadCoordinator.java | 6 ++-- ...ordOffsetObject.java => RecordOffset.java} | 4 +-- .../com/teragrep/cfe_39/CombinedFullTest.java | 23 ++++++++------- .../java/com/teragrep/cfe_39/HdfsTest.java | 16 +++++------ .../java/com/teragrep/cfe_39/PruneTest.java | 12 ++++---- 12 files changed, 71 insertions(+), 72 deletions(-) rename src/main/java/com/teragrep/cfe_39/consumers/kafka/{NullOffsetObject.java => NullOffset.java} (95%) rename src/main/java/com/teragrep/cfe_39/consumers/kafka/{OffsetObject.java => Offset.java} (96%) rename src/main/java/com/teragrep/cfe_39/consumers/kafka/{RecordOffsetObject.java => RecordOffset.java} (94%) diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java index 0ecd965b..5eef3112 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java @@ -40,7 +40,7 @@ // The mock consumer is activated for testing using the configuration file: readerKafkaProperties.getProperty("useMockKafkaConsumer", "false") -public class DatabaseOutput implements Consumer> { +public class DatabaseOutput implements Consumer> { private static final Logger LOGGER = LoggerFactory.getLogger(DatabaseOutput.class); private final RFC5424Frame rfc5424Frame = new RFC5424Frame(false); @@ -107,7 +107,7 @@ public class DatabaseOutput implements Consumer> { } // Checks that the filesize stays under the defined maximum file size. If the file is about to go over target limit commits the file to HDFS and returns true, otherwise does nothing and returns false. - boolean committedToHdfs(long fileSize, RecordOffsetObject recordOffsetObject) { + boolean committedToHdfs(long fileSize, RecordOffset recordOffsetObject) { try { // If the syslogAvroWriter is already initialized, check the filesize so it doesn't go above maximumFileSize. if (fileSize > maximumFileSize) { @@ -148,7 +148,7 @@ private long rfc3339ToEpoch(ZonedDateTime zonedDateTime) { // AVRO-file with a path/name that starts with topic_name/0.X should only contain records from the 0th partition of topic named topic_name, topic_name/1.X should only contain records from 1st partition, etc. // AVRO-files are created dynamically, thus it is not known which record (and its offset) is written to the file last before committing it to HDFS. The final name for the HDFS file is decided only when the file is committed to HDFS. @Override - public void accept(List recordOffsetObjectList) { + public void accept(List recordOffsetObjectList) { long thisTime = Instant.now().toEpochMilli(); long ftook = thisTime - lastTimeCalled; topicCounter.setKafkaLatency(ftook); @@ -160,10 +160,10 @@ public void accept(List recordOffsetObjectList) { // When the file size is about to go above 64M, commit the file into HDFS using the latest topic/partition/offset values as the filename and start fresh with a new empty AVRO-file. // Serialize the object that was going to make the file go above 64M into the now empty AVRO-file and continue the loop. // TODO: If the prod-environment recordOffsetObjectList ordering is different from what it is in the test environment, add a function that reorders the list based on partition and offset (or better yet, make several AVRO-files that are being used at the same time rather than doing it one AVRO-file at a time as the offset ordering within partitions should always be correct in all scenarios). - OffsetObject lastObject = new NullOffsetObject(); // Abstract OffsetObject class, set to null object before initializing as RecordOffsetObject. + Offset lastObject = new NullOffset(); // Abstract OffsetObject class, set to null object before initializing as RecordOffsetObject. long start = Instant.now().toEpochMilli(); // Starts measuring performance here. Measures how long it takes to process the whole recordOffsetObjectList. // This loop goes through all the records of the mock data in a single session. - for (RecordOffsetObject recordOffsetObject : recordOffsetObjectList) { + for (RecordOffset recordOffsetObject : recordOffsetObjectList) { // Initializing syslogAvroWriter and lastObject. if (syslogAvroWriter == null && lastObject.isNull()) { try { @@ -181,7 +181,7 @@ public void accept(List recordOffsetObjectList) { try { if (lastObject.getTopic().equals(recordOffsetObject.getTopic()) & lastObject.getPartition().equals(recordOffsetObject.getPartition())) { // Records left to consume in the current partition. - boolean b = committedToHdfs(syslogAvroWriter.getFileSize(), (RecordOffsetObject) lastObject); + boolean b = committedToHdfs(syslogAvroWriter.getFileSize(), (RecordOffset) lastObject); if (b) { LOGGER.debug("Target file size reached, file {} stored to {} in HDFS", syslogFile.getName(), lastObject.getTopic() + "/" + lastObject.getPartition() + "." + lastObject.getOffset()); } else { @@ -190,7 +190,7 @@ public void accept(List recordOffsetObjectList) { } else { // Previous partition was fully consumed. Commit file to HDFS and create a new AVRO-file. syslogAvroWriter.close(); - HDFSWriter writer = new HDFSWriter(config, (RecordOffsetObject) lastObject); + HDFSWriter writer = new HDFSWriter(config, (RecordOffset) lastObject); writer.commit(syslogFile, epochMicros_last); // This part defines a new empty file to which the new AVRO-serialized records are stored until it again hits the 64M size limit. @@ -259,7 +259,7 @@ public void accept(List recordOffsetObjectList) { } // Check if there is still room in syslogAvroWriter for another syslogRecord. Commit syslogAvroWriter to HDFS if no room left, emptying it out in the process. // boolean b = committedToHdfs(approximatedSize + capacity, lastObject); // FIXME: approximatedSize is not working properly without the use of flush() after append. File sizes are all over the place. - boolean b = committedToHdfs(syslogAvroWriter.getFileSize() + capacity, (RecordOffsetObject) lastObject); + boolean b = committedToHdfs(syslogAvroWriter.getFileSize() + capacity, (RecordOffset) lastObject); if (b) { LOGGER.debug("Target file size reached, file {} stored to {} in HDFS", syslogFile.getName(), lastObject.getTopic()+"/"+lastObject.getPartition()+"."+lastObject.getOffset()); }else { @@ -282,7 +282,7 @@ public void accept(List recordOffsetObjectList) { try { if (syslogAvroWriter != null && !lastObject.isNull()) { syslogAvroWriter.close(); - try (HDFSWriter writer = new HDFSWriter(config, (RecordOffsetObject) lastObject)) { + try (HDFSWriter writer = new HDFSWriter(config, (RecordOffset) lastObject)) { writer.commit(syslogFile, epochMicros_last); // commits the final AVRO-file to HDFS. } } diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSPrune.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSPrune.java index 2268183a..4140bde8 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSPrune.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSPrune.java @@ -37,7 +37,7 @@ public class HDFSPrune { private static final Logger LOGGER = LoggerFactory.getLogger(HDFSPrune.class); private Config config; private final FileSystem fs; - private Path newFolderPath; + private Path newDirectoryPath; private long cutOffEpoch; private final boolean useMockKafkaConsumer; // test-mode switch @@ -66,12 +66,12 @@ public HDFSPrune(Config config, String topicName) throws IOException { //Get the filesystem - HDFS fs = FileSystem.get(URI.create(hdfsuri), conf); - //==== Create folder if not exists + //==== Create directory if not exists Path workingDir = fs.getWorkingDirectory(); - newFolderPath = new Path(path); - if (!fs.exists(newFolderPath)) { + newDirectoryPath = new Path(path); + if (!fs.exists(newDirectoryPath)) { // Create new Directory - fs.mkdirs(newFolderPath); + fs.mkdirs(newDirectoryPath); LOGGER.info("Path {} created.", path); } }else { @@ -109,12 +109,12 @@ public HDFSPrune(Config config, String topicName) throws IOException { // filesystem for HDFS access is set here fs = FileSystem.get(conf); - //==== Create folder if not exists + //==== Create directory if not exists Path workingDir = fs.getWorkingDirectory(); - newFolderPath = new Path(path); - if (!fs.exists(newFolderPath)) { + newDirectoryPath = new Path(path); + if (!fs.exists(newDirectoryPath)) { // Create new Directory - fs.mkdirs(newFolderPath); + fs.mkdirs(newDirectoryPath); LOGGER.info("Path {} created.", path); } } @@ -124,7 +124,7 @@ public HDFSPrune(Config config, String topicName) throws IOException { public void prune() throws IOException { // Fetch the filestatuses of HDFS files. - FileStatus[] fileStatuses = fs.listStatus(new Path(newFolderPath + "/")); + FileStatus[] fileStatuses = fs.listStatus(new Path(newDirectoryPath + "/")); if (fileStatuses.length > 0) { for (FileStatus a : fileStatuses) { // Delete old files @@ -134,7 +134,7 @@ public void prune() throws IOException { } } } else { - LOGGER.info("No files found in folder {}", new Path(newFolderPath + "/")); + LOGGER.info("No files found in directory {}", new Path(newDirectoryPath + "/")); } } } diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java index 76013528..13da6255 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java @@ -45,7 +45,7 @@ public class HDFSWriter implements AutoCloseable{ // Create files as whole but stream the contents into them. Avro files 'flush' must be called as few times as possible. Check memory usage impact // Later make sure to check the avro file flush issue where the file size is all over the place if flush is not used after every append to the file. - public HDFSWriter(Config config, RecordOffsetObject lastObject) throws IOException { + public HDFSWriter(Config config, RecordOffset lastObject) throws IOException { // Check for testmode from config. Properties readerKafkaProperties = config.getKafkaConsumerProperties(); @@ -58,7 +58,7 @@ public HDFSWriter(Config config, RecordOffsetObject lastObject) throws IOExcepti hdfsuri = config.getHdfsuri(); // Get from config. // The filepath should be something like hdfs:///opt/teragrep/cfe_39/srv/topic_name/0.12345 where 12345 is offset and 0 the partition. - // In other words the folder named topic_name holds files that are named and arranged based on partition and the partition's offset. Every partition has its own set of unique offset values. + // In other words the directory named topic_name holds files that are named and arranged based on partition and the partition's offset. Every partition has its own set of unique offset values. // These values should be fetched from config and other input parameters (topic+partition+offset). path = config.getHdfsPath()+"/"+lastObject.topic; fileName = lastObject.partition+"."+lastObject.offset; // filename should be constructed from partition and offset. @@ -86,9 +86,9 @@ public HDFSWriter(Config config, RecordOffsetObject lastObject) throws IOExcepti hdfsuri = config.getHdfsuri(); // Get from config. // The filepath should be something like hdfs:///opt/teragrep/cfe_39/srv/topic_name/0.12345 where 12345 is offset and 0 the partition. - // In other words the folder named topic_name holds files that are named and arranged based on partition and the partition's offset. Every partition has its own set of unique offset values. + // In other words the directory named topic_name holds files that are named and arranged based on partition and the partition's offset. Every partition has its own set of unique offset values. // The values are fetched from config and input parameters (topic+partition+offset). - path = config.getHdfsPath() + "/" + lastObject.topic; // folder path is constructed from HdfsPath and topic name. + path = config.getHdfsPath() + "/" + lastObject.topic; // directory path is constructed from HdfsPath and topic name. fileName = lastObject.partition + "." + lastObject.offset; // filename should be constructed from partition and offset. @@ -134,20 +134,20 @@ public void commit(File syslogFile, long lastEpochMicros) { // CODE FOR TEST-MODE GOES HERE! //Get the filesystem - HDFS try { - //==== Create folder if not exists + //==== Create directory if not exists Path workingDir = fs.getWorkingDirectory(); // Sets the directory where the data should be stored, if the directory doesn't exist then it's created. - Path newFolderPath = new Path(path); - if (!fs.exists(newFolderPath)) { + Path newDirectoryPath = new Path(path); + if (!fs.exists(newDirectoryPath)) { // Create new Directory - fs.mkdirs(newFolderPath); + fs.mkdirs(newDirectoryPath); LOGGER.debug("Path {} created.", path); } //==== Write file LOGGER.debug("Begin Write file into hdfs"); //Create a path - Path hdfswritepath = new Path(newFolderPath + "/" + fileName); // filename should be set according to the requirements: 0.12345 where 0 is Kafka partition and 12345 is Kafka offset. + Path hdfswritepath = new Path(newDirectoryPath + "/" + fileName); // filename should be set according to the requirements: 0.12345 where 0 is Kafka partition and 12345 is Kafka offset. if (fs.exists(hdfswritepath)) { throw new RuntimeException("File " + fileName + " already exists"); } @@ -179,20 +179,20 @@ public void commit(File syslogFile, long lastEpochMicros) { }else { //Get the filesystem - HDFS try { - //==== Create folder if not exists + //==== Create directory if not exists Path workingDir = fs.getWorkingDirectory(); // Sets the directory where the data should be stored, if the directory doesn't exist then it's created. - Path newFolderPath = new Path(path); - if (!fs.exists(newFolderPath)) { + Path newDirectoryPath = new Path(path); + if (!fs.exists(newDirectoryPath)) { // Create new Directory - fs.mkdirs(newFolderPath); + fs.mkdirs(newDirectoryPath); LOGGER.debug("Path {} created.", path); } //==== Write file LOGGER.debug("Begin Write file into hdfs"); //Create a path - Path hdfswritepath = new Path(newFolderPath + "/" + fileName); // filename should be set according to the requirements: 0.12345 where 0 is Kafka partition and 12345 is Kafka offset. + Path hdfswritepath = new Path(newDirectoryPath + "/" + fileName); // filename should be set according to the requirements: 0.12345 where 0 is Kafka partition and 12345 is Kafka offset. if (fs.exists(hdfswritepath)) { throw new RuntimeException("File " + fileName + " already exists"); } diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java index 37243c15..fc04f930 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java @@ -139,7 +139,7 @@ private void createReader(String topic, List listPartitionInfo, L // DatabaseOutput handles transferring the consumed data to storage (S3, mariadb, HDFS, etc.) // Kafka offset tracking must be included here. // Topic is figured out in topicScan so the offsets for the topic should be figured out here. - Consumer> output = new DatabaseOutput( + Consumer> output = new DatabaseOutput( config, // Configuration settings topic, // String, the name of the topic durationStatistics, // RuntimeStatistics object from metrics diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReader.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReader.java index 9f3d3394..ed14887f 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReader.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReader.java @@ -29,10 +29,10 @@ public class KafkaReader implements AutoCloseable { final Logger LOGGER = LoggerFactory.getLogger(KafkaReader.class); private Iterator> kafkaRecordsIterator = Collections.emptyIterator(); private final Consumer kafkaConsumer; - private final java.util.function.Consumer> callbackFunction; + private final java.util.function.Consumer> callbackFunction; public KafkaReader( - Consumer kafkaConsumer, java.util.function.Consumer> callbackFunction) { + Consumer kafkaConsumer, java.util.function.Consumer> callbackFunction) { this.kafkaConsumer = kafkaConsumer; this.callbackFunction = callbackFunction; } @@ -48,11 +48,11 @@ public void read() { kafkaRecordsIterator = kafkaRecords.iterator(); } - List recordOffsetObjectList = new ArrayList<>(); + List recordOffsetObjectList = new ArrayList<>(); while (kafkaRecordsIterator.hasNext()) { ConsumerRecord record = kafkaRecordsIterator.next(); LOGGER.debug("adding from offset: " + record.offset()); - recordOffsetObjectList.add(new RecordOffsetObject(record.topic(), record.partition(), record.offset(), record.value())); + recordOffsetObjectList.add(new RecordOffset(record.topic(), record.partition(), record.offset(), record.value())); /* // SKIPPING IDEMPOTENT CONSUMER IMPLEMENTATION FOR NOW! boolean checkStuff = checkIfProcessed(record.topic(), record.partition(), record.offset()); // Create checkIfProcessed method. Checks if the record has already been processed and stored in HDFS. diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/NullOffsetObject.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/NullOffset.java similarity index 95% rename from src/main/java/com/teragrep/cfe_39/consumers/kafka/NullOffsetObject.java rename to src/main/java/com/teragrep/cfe_39/consumers/kafka/NullOffset.java index fe7166af..6727db68 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/NullOffsetObject.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/NullOffset.java @@ -16,7 +16,7 @@ */ package com.teragrep.cfe_39.consumers.kafka; -public class NullOffsetObject extends OffsetObject{ +public class NullOffset extends Offset{ @Override public boolean isNull() { diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/OffsetObject.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/Offset.java similarity index 96% rename from src/main/java/com/teragrep/cfe_39/consumers/kafka/OffsetObject.java rename to src/main/java/com/teragrep/cfe_39/consumers/kafka/Offset.java index cd123e0a..4a931166 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/OffsetObject.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/Offset.java @@ -16,7 +16,7 @@ */ package com.teragrep.cfe_39.consumers.kafka; -public abstract class OffsetObject { +public abstract class Offset { protected String topic; protected Integer partition; protected Long offset; diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinator.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinator.java index 156bcf6a..69f8df0e 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinator.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinator.java @@ -31,13 +31,13 @@ public class ReadCoordinator implements Runnable { private final String queueTopic; private final Properties readerKafkaProperties; - private final Consumer> callbackFunction; + private final Consumer> callbackFunction; private boolean run = true; public ReadCoordinator( String queueTopic, Properties readerKafkaProperties, - Consumer> callbackFunction) + Consumer> callbackFunction) { this.queueTopic = queueTopic; this.readerKafkaProperties = readerKafkaProperties; @@ -46,7 +46,7 @@ public ReadCoordinator( private KafkaReader createKafkaReader(Properties readerKafkaProperties, String topic, - Consumer> callbackFunction, + Consumer> callbackFunction, boolean useMockKafkaConsumer) { org.apache.kafka.clients.consumer.Consumer kafkaConsumer; diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/RecordOffsetObject.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/RecordOffset.java similarity index 94% rename from src/main/java/com/teragrep/cfe_39/consumers/kafka/RecordOffsetObject.java rename to src/main/java/com/teragrep/cfe_39/consumers/kafka/RecordOffset.java index 69b736f6..b566c39e 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/RecordOffsetObject.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/RecordOffset.java @@ -18,9 +18,9 @@ package com.teragrep.cfe_39.consumers.kafka; // This is the class for handling the Kafka record topic/partition/offset data that are required for HDFS storage. -public class RecordOffsetObject extends OffsetObject{ +public class RecordOffset extends Offset{ - public RecordOffsetObject( + public RecordOffset( String topic, int partition, long offset, diff --git a/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java b/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java index 92966d0d..8d98e056 100644 --- a/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java +++ b/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java @@ -82,7 +82,6 @@ public void kafkaAndAvroFullTest() throws InterruptedException { } } - @Test public void hdfsPruneTest() throws IOException { // Check that the files were properly written to HDFS with a read test. @@ -102,12 +101,12 @@ public void hdfsPruneTest() throws IOException { //Get the filesystem - HDFS FileSystem fs = FileSystem.get(URI.create(hdfsuri), conf); - //==== Create folder if not exists + //==== Create directory if not exists Path workingDir=fs.getWorkingDirectory(); - Path newFolderPath= new Path(path); - if(!fs.exists(newFolderPath)) { + Path newDirectoryPath= new Path(path); + if(!fs.exists(newDirectoryPath)) { // Create new Directory - fs.mkdirs(newFolderPath); + fs.mkdirs(newDirectoryPath); LOGGER.info("Path {} created.", path); } @@ -122,7 +121,7 @@ public void hdfsPruneTest() throws IOException { // Another method for pruning aside using avro-mapred is to use modification timestamp of the avro-file stored in HDFS: // fs.setTimes(new Path(path+"/"+0.8), Long.parseUnsignedLong("1675930598000"), -1); // where mtime is modification time and atime is access time. -1 as input parameter leaves the original atime/mtime value as is. - FileStatus[] fileStatuses = fs.listStatus(new Path(newFolderPath + "/")); + FileStatus[] fileStatuses = fs.listStatus(new Path(newDirectoryPath + "/")); long count = Arrays.stream(fileStatuses).count(); if (count != 0) { if (config.getPruneOffset() != 157784760000L) { @@ -167,17 +166,17 @@ public void hdfsReadCheck() throws IOException { //Get the filesystem - HDFS FileSystem fs = FileSystem.get(URI.create(hdfsuri), conf); - //==== Create folder if not exists + //==== Create directory if not exists Path workingDir=fs.getWorkingDirectory(); - Path newFolderPath= new Path(path); - if(!fs.exists(newFolderPath)) { + Path newDirectoryPath= new Path(path); + if(!fs.exists(newDirectoryPath)) { // Create new Directory - fs.mkdirs(newFolderPath); + fs.mkdirs(newDirectoryPath); LOGGER.info("Path {} created.", path); } // This is the HDFS write path for the files: - // Path hdfswritepath = new Path(newFolderPath + "/" + fileName); where newFolderPath is config.getHdfsPath() + "/" + lastObject.topic; and filename is lastObject.partition+"."+lastObject.offset; + // Path hdfswritepath = new Path(newDirectoryPath + "/" + fileName); where newDirectoryPath is config.getHdfsPath() + "/" + lastObject.topic; and filename is lastObject.partition+"."+lastObject.offset; // Create the list of files to read from HDFS. Test setup is created so each of the 0-9 partitions will have 2 files with offsets of 8 and 13. List filenameList = new ArrayList<>(); @@ -191,7 +190,7 @@ public void hdfsReadCheck() throws IOException { //==== Read files LOGGER.info("Read file into hdfs"); //Create a path - Path hdfsreadpath = new Path(newFolderPath + "/" + fileName); // The path should be the same that was used in writing the file to HDFS. + Path hdfsreadpath = new Path(newDirectoryPath + "/" + fileName); // The path should be the same that was used in writing the file to HDFS. //Init input stream FSDataInputStream inputStream = fs.open(hdfsreadpath); //The data is in AVRO-format, so it can't be read as a string. diff --git a/src/test/java/com/teragrep/cfe_39/HdfsTest.java b/src/test/java/com/teragrep/cfe_39/HdfsTest.java index 4f660d1e..eed5f810 100644 --- a/src/test/java/com/teragrep/cfe_39/HdfsTest.java +++ b/src/test/java/com/teragrep/cfe_39/HdfsTest.java @@ -4,7 +4,7 @@ import com.teragrep.cfe_39.consumers.kafka.DatabaseOutput; import com.teragrep.cfe_39.consumers.kafka.HDFSWriter; import com.teragrep.cfe_39.consumers.kafka.KafkaController; -import com.teragrep.cfe_39.consumers.kafka.RecordOffsetObject; +import com.teragrep.cfe_39.consumers.kafka.RecordOffset; import org.apache.avro.file.DataFileReader; import org.apache.avro.file.DataFileStream; import org.apache.avro.io.DatumReader; @@ -136,7 +136,7 @@ public void hdfsWriteTest() { } assert lastRecord != null; - RecordOffsetObject lastObject = new RecordOffsetObject("testConsumerTopic", Integer.parseInt(lastRecord.getPartition().toString()), lastRecord.getOffset(), null); // Fetch input parameters from the lastRecord SyslogRecord-object. + RecordOffset lastObject = new RecordOffset("testConsumerTopic", Integer.parseInt(lastRecord.getPartition().toString()), lastRecord.getOffset(), null); // Fetch input parameters from the lastRecord SyslogRecord-object. LOGGER.debug("\n"+"Last record in the " + syslogFile.getName() + " file:" + "\ntopic: " + lastObject.getTopic() + "\npartition: " + lastObject.getPartition() + "\noffset: " + lastObject.getOffset()); try (HDFSWriter writer = new HDFSWriter(config, lastObject)) { writer.commit(syslogFile, -1L); // commits the final AVRO-file to HDFS. @@ -179,22 +179,22 @@ public void hdfsReadCheck(String testConsumerTopic, int partition, long offset) //Get the filesystem - HDFS FileSystem fs = FileSystem.get(URI.create(hdfsuri), conf); - //==== Create folder if not exists + //==== Create directory if not exists Path workingDir=fs.getWorkingDirectory(); - Path newFolderPath= new Path(path); - if(!fs.exists(newFolderPath)) { + Path newDirectoryPath= new Path(path); + if(!fs.exists(newDirectoryPath)) { // Create new Directory - fs.mkdirs(newFolderPath); + fs.mkdirs(newDirectoryPath); // logger.info("Path "+path+" created."); } // This is the HDFS write path for the files: - // Path hdfswritepath = new Path(newFolderPath + "/" + fileName); where newFolderPath is config.getHdfsPath() + "/" + lastObject.topic; and filename is lastObject.partition+"."+lastObject.offset; + // Path hdfswritepath = new Path(newDirectoryPath + "/" + fileName); where newDirectoryPath is config.getHdfsPath() + "/" + lastObject.topic; and filename is lastObject.partition+"."+lastObject.offset; //==== Read files // logger.info("Read file into hdfs"); //Create a path - Path hdfsreadpath = new Path(newFolderPath + "/" + fileName); // The path should be the same that was used in writing the file to HDFS. + Path hdfsreadpath = new Path(newDirectoryPath + "/" + fileName); // The path should be the same that was used in writing the file to HDFS. //Init input stream FSDataInputStream inputStream = fs.open(hdfsreadpath); //The data is in AVRO-format, so it can't be read as a string. diff --git a/src/test/java/com/teragrep/cfe_39/PruneTest.java b/src/test/java/com/teragrep/cfe_39/PruneTest.java index 508cb1c3..b322fa8f 100644 --- a/src/test/java/com/teragrep/cfe_39/PruneTest.java +++ b/src/test/java/com/teragrep/cfe_39/PruneTest.java @@ -32,7 +32,7 @@ // This class should be compiled into a jar-file that is then sent to the hadoop cluster for running the job when needed. Maven should be configured to do the jar-packaging etc. /* The records are stored inside files that are 64MB in size and named depending on which Kafka partition offset the last stored record belongs to. -In other words the files are inside topic_name-folder and there are at least one file per partition, depending on the load size of records that are fetched from Kafka topics. +In other words the files are inside topic_name-directory and there are at least one file per partition, depending on the load size of records that are fetched from Kafka topics. The AVRO-files that hold the records can house over 240,000 records each (at 64MB), but that is just an estimate because the record sizes vary widely. This means that the partition offsets are not guaranteed to be same for all the files in a topic. In other words the same number of records could be distributed between 3 files on one partition and 2 files on another partition because of the different individual record sizes. @@ -52,9 +52,9 @@ But in any case the pruning should include deleting AVRO-files that hold only outdated records that should be pruned. The handling of the leftover garbage records can be handled later in the MapReduce of the datasource component queries. */ -// The main function that will call for pruning will know the topic name (aka. folder path). The pruning will be done in folder basis, aka. in topic basis, so tracking the topic name is not important for the MapReduce as the input path already contains the topic name. +// The main function that will call for pruning will know the topic name (aka. directory path). The pruning will be done in directory basis, aka. in topic basis, so tracking the topic name is not important for the MapReduce as the input path already contains the topic name. // Instead, the partition and offset values together with timestamp are important for pruning. The MapReduce function should create a list of key-value pairs where key is the partition+offset and value is the timestamp, where timestamp is smaller than the cutoff_epoch defined by input arguments. -// The pruning of old records can be called in KafkaController.java row 112, using the activeTopics list as a input argument for topic names. This way the records are pruned every time new ones are added. Make sure there are no concurrency issues with the HDFS writer. Most likely there is a need for pruning-controller class that will manage the folder/topic scanning etc. +// The pruning of old records can be called in KafkaController.java row 112, using the activeTopics list as a input argument for topic names. This way the records are pruned every time new ones are added. Make sure there are no concurrency issues with the HDFS writer. Most likely there is a need for pruning-controller class that will manage the directory/topic scanning etc. public class PruneTest extends Configured implements Tool { static long cutoff_epoch; // TimestampMapper takes a SyslogRecord as input and outputs a key-value pair of record partition+"."+offset and timestamp of the record. @@ -105,7 +105,7 @@ public int run(String[] args) throws Exception { Job job = Job.getInstance(conf, "timestamp prune"); job.setJarByClass(PruneTest.class); - SequenceFileInputFormat.setInputPaths(job, new Path(args[0])); // The input path should be the folder where the AVRO-files are held. setInputPaths can take either folder or file as input, not sure if using folder has the same effect as having a list of files. + SequenceFileInputFormat.setInputPaths(job, new Path(args[0])); // The input path should be the directory where the AVRO-files are held. setInputPaths can take either directory or file as input, not sure if using directory has the same effect as having a list of files. FileOutputFormat.setOutputPath(job, new Path(args[1])); // Output path is where the results of the MapReduce are stored. job.setInputFormatClass(AvroKeyInputFormat.class); @@ -122,9 +122,9 @@ public int run(String[] args) throws Exception { return (job.waitForCompletion(true) ? 0 : 1); } - // Set input folder to be the topic folder. + // Set input directory to be the topic directory. public static void main(String[] args) throws Exception { - int res = ToolRunner.run(new PruneTest(), args); // arg1 is and arg2 is , output path should be a new HDFS folder that does not exist and input path should be the HDFS folder with AVRO-files that we have generated in tests. + int res = ToolRunner.run(new PruneTest(), args); // arg1 is and arg2 is , output path should be a new HDFS directory that does not exist and input path should be the HDFS directory with AVRO-files that we have generated in tests. System.exit(res); } } From 1b845bc42f0e188370337e997d20cbf2ba439074 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Tue, 14 May 2024 14:16:00 +0300 Subject: [PATCH 056/146] Added maven compiler plugin version tag, simplified versioning for hadoop in pom.xml. --- pom.xml | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/pom.xml b/pom.xml index a7d217be..f87cfd4c 100644 --- a/pom.xml +++ b/pom.xml @@ -18,6 +18,7 @@ 4.2.8 0.16.0 + 3.3.6 @@ -41,6 +42,7 @@ org.apache.maven.plugins maven-compiler-plugin + 3.8.1 1.8 1.8 @@ -134,26 +136,26 @@ org.apache.hadoop hadoop-client - 3.3.6 + ${hadoop.version} org.apache.hadoop hadoop-minicluster - 3.3.6 + ${hadoop.version} org.apache.hadoop hadoop-common - 3.3.6 + ${hadoop.version} org.apache.hadoop hadoop-hdfs - 3.3.6 + ${hadoop.version} test From 3cb7adc68063cff622e69807518b39869089cbd0 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Tue, 14 May 2024 14:26:33 +0300 Subject: [PATCH 057/146] Renamed HDFSWriter.java to HDFSWrite.java. --- .../cfe_39/consumers/kafka/DatabaseOutput.java | 6 +++--- .../kafka/{HDFSWriter.java => HDFSWrite.java} | 14 ++++++++------ src/test/java/com/teragrep/cfe_39/HdfsTest.java | 8 ++------ 3 files changed, 13 insertions(+), 15 deletions(-) rename src/main/java/com/teragrep/cfe_39/consumers/kafka/{HDFSWriter.java => HDFSWrite.java} (95%) diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java index 5eef3112..ded4cc52 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java @@ -115,7 +115,7 @@ boolean committedToHdfs(long fileSize, RecordOffset recordOffsetObject) { // This part closes the writing of now "complete" AVRO-file and stores the file to HDFS. syslogAvroWriter.close(); - try (HDFSWriter writer = new HDFSWriter(config, recordOffsetObject)) { + try (HDFSWrite writer = new HDFSWrite(config, recordOffsetObject)) { writer.commit(syslogFile, epochMicros_last); // commits the final AVRO-file to HDFS. } @@ -190,7 +190,7 @@ public void accept(List recordOffsetObjectList) { } else { // Previous partition was fully consumed. Commit file to HDFS and create a new AVRO-file. syslogAvroWriter.close(); - HDFSWriter writer = new HDFSWriter(config, (RecordOffset) lastObject); + HDFSWrite writer = new HDFSWrite(config, (RecordOffset) lastObject); writer.commit(syslogFile, epochMicros_last); // This part defines a new empty file to which the new AVRO-serialized records are stored until it again hits the 64M size limit. @@ -282,7 +282,7 @@ public void accept(List recordOffsetObjectList) { try { if (syslogAvroWriter != null && !lastObject.isNull()) { syslogAvroWriter.close(); - try (HDFSWriter writer = new HDFSWriter(config, (RecordOffset) lastObject)) { + try (HDFSWrite writer = new HDFSWrite(config, (RecordOffset) lastObject)) { writer.commit(syslogFile, epochMicros_last); // commits the final AVRO-file to HDFS. } } diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java similarity index 95% rename from src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java rename to src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java index 13da6255..05667105 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWriter.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java @@ -32,9 +32,9 @@ import java.nio.file.Paths; import java.util.Properties; -public class HDFSWriter implements AutoCloseable{ +public class HDFSWrite implements AutoCloseable{ - private static final Logger LOGGER = LoggerFactory.getLogger(HDFSWriter.class); + private static final Logger LOGGER = LoggerFactory.getLogger(HDFSWrite.class); private final String fileName; private final String path; private final FileSystem fs; @@ -45,7 +45,7 @@ public class HDFSWriter implements AutoCloseable{ // Create files as whole but stream the contents into them. Avro files 'flush' must be called as few times as possible. Check memory usage impact // Later make sure to check the avro file flush issue where the file size is all over the place if flush is not used after every append to the file. - public HDFSWriter(Config config, RecordOffset lastObject) throws IOException { + public HDFSWrite(Config config, RecordOffset lastObject) throws IOException { // Check for testmode from config. Properties readerKafkaProperties = config.getKafkaConsumerProperties(); @@ -141,15 +141,17 @@ public void commit(File syslogFile, long lastEpochMicros) { if (!fs.exists(newDirectoryPath)) { // Create new Directory fs.mkdirs(newDirectoryPath); - LOGGER.debug("Path {} created.", path); + LOGGER.info("Path {} created.", path); } //==== Write file LOGGER.debug("Begin Write file into hdfs"); //Create a path - Path hdfswritepath = new Path(newDirectoryPath + "/" + fileName); // filename should be set according to the requirements: 0.12345 where 0 is Kafka partition and 12345 is Kafka offset. + Path hdfswritepath = new Path(newDirectoryPath.toString() + "/" + fileName); // filename should be set according to the requirements: 0.12345 where 0 is Kafka partition and 12345 is Kafka offset. if (fs.exists(hdfswritepath)) { throw new RuntimeException("File " + fileName + " already exists"); + } else { + LOGGER.info("Path {} doesn't exist.", path); } /*//Init output stream @@ -171,7 +173,7 @@ public void commit(File syslogFile, long lastEpochMicros) { // updateTimestamp(hdfswritepath, lastEpochMicros); LOGGER.debug("End Write file into hdfs"); boolean delete = syslogFile.delete(); // deletes the avro-file from the local disk now that it has been committed to HDFS. - LOGGER.debug("\nFile committed to HDFS, file writepath should be: {}\n", hdfswritepath.toString()); + LOGGER.info("\nFile committed to HDFS, file writepath should be: {}\n", hdfswritepath.toString()); } catch (IOException e) { throw new RuntimeException(e); diff --git a/src/test/java/com/teragrep/cfe_39/HdfsTest.java b/src/test/java/com/teragrep/cfe_39/HdfsTest.java index eed5f810..a067c7cb 100644 --- a/src/test/java/com/teragrep/cfe_39/HdfsTest.java +++ b/src/test/java/com/teragrep/cfe_39/HdfsTest.java @@ -1,8 +1,7 @@ package com.teragrep.cfe_39; import com.teragrep.cfe_39.avro.SyslogRecord; -import com.teragrep.cfe_39.consumers.kafka.DatabaseOutput; -import com.teragrep.cfe_39.consumers.kafka.HDFSWriter; +import com.teragrep.cfe_39.consumers.kafka.HDFSWrite; import com.teragrep.cfe_39.consumers.kafka.KafkaController; import com.teragrep.cfe_39.consumers.kafka.RecordOffset; import org.apache.avro.file.DataFileReader; @@ -16,10 +15,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.MiniDFSCluster; -import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -138,7 +134,7 @@ public void hdfsWriteTest() { assert lastRecord != null; RecordOffset lastObject = new RecordOffset("testConsumerTopic", Integer.parseInt(lastRecord.getPartition().toString()), lastRecord.getOffset(), null); // Fetch input parameters from the lastRecord SyslogRecord-object. LOGGER.debug("\n"+"Last record in the " + syslogFile.getName() + " file:" + "\ntopic: " + lastObject.getTopic() + "\npartition: " + lastObject.getPartition() + "\noffset: " + lastObject.getOffset()); - try (HDFSWriter writer = new HDFSWriter(config, lastObject)) { + try (HDFSWrite writer = new HDFSWrite(config, lastObject)) { writer.commit(syslogFile, -1L); // commits the final AVRO-file to HDFS. } catch (IOException e) { throw new RuntimeException(e); From 094c21c75505098142f0fbd22607bbaa8c149e33 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Tue, 14 May 2024 14:49:52 +0300 Subject: [PATCH 058/146] Added HDFSRead.java for fetching offsets of the latest kafka records committed to HDFS database. --- .../cfe_39/consumers/kafka/HDFSRead.java | 172 ++++++++++++++++++ .../consumers/kafka/KafkaController.java | 12 ++ 2 files changed, 184 insertions(+) create mode 100644 src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSRead.java diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSRead.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSRead.java new file mode 100644 index 00000000..8fd83f64 --- /dev/null +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSRead.java @@ -0,0 +1,172 @@ +package com.teragrep.cfe_39.consumers.kafka; + +import com.teragrep.cfe_39.Config; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.*; +import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.kafka.common.TopicPartition; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.net.URI; +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; + +public final class HDFSRead implements AutoCloseable { + // Maps out the latest offset for all the topic partitions available in HDFS. + // The offset map can then be used for kafka consumer seek() method, which will add the idempotent functionality to the consumer. + // Also, because this class should be called outside the loops that generate the consumer groups it should be lightweight to run. + + private static final Logger LOGGER = LoggerFactory.getLogger(HDFSRead.class); + private final FileSystem fs; + private final boolean useMockKafkaConsumer; // test-mode switch + private final Configuration conf; + private final String hdfsuri; + private static String topicsRegexString = null; + private final String path; + + public HDFSRead(Config config) throws IOException { + // Check for testmode from config. + Properties readerKafkaProperties = config.getKafkaConsumerProperties(); + this.useMockKafkaConsumer = Boolean.parseBoolean( + readerKafkaProperties.getProperty("useMockKafkaConsumer", "false") + ); + + if (useMockKafkaConsumer) { + // Code for initializing the class in test mode without kerberos. + hdfsuri = config.getHdfsuri(); // Get from config. + path = config.getHdfsPath(); + + // ====== Init HDFS File System Object + conf = new Configuration(); + // Set FileSystem URI + conf.set("fs.defaultFS", hdfsuri); + // Because of Maven + conf.set("fs.hdfs.impl", DistributedFileSystem.class.getName()); + conf.set("fs.file.impl", LocalFileSystem.class.getName()); + // Set HADOOP user here, Kerberus parameters most likely needs to be added here too. + System.setProperty("HADOOP_USER_NAME", "hdfs"); + System.setProperty("hadoop.home.dir", "/"); + // filesystem for HDFS access is set here + try { + fs = FileSystem.get(URI.create(hdfsuri), conf); + } catch (IOException e) { + throw new RuntimeException(e); + } + + /*//==== Create directory if not exists + Path workingDir=fs.getWorkingDirectory(); + Path newDirectoryPath= new Path(path); + if(!fs.exists(newDirectoryPath)) { + // Create new Directory + fs.mkdirs(newDirectoryPath); + LOGGER.info("Path {} created.", path); + }*/ + + + }else { + // Code for initializing the class with kerberos. + hdfsuri = config.getHdfsuri(); // Get from config.' + path = config.getHdfsPath(); + + + // Set HADOOP user here, Kerberus parameters most likely needs to be added here too. + // System.setProperty("HADOOP_USER_NAME", "hdfs"); // Not needed because user authentication is done by kerberos? + // System.setProperty("hadoop.home.dir", "/"); // Not needed because user authentication is done by kerberos? + + // set kerberos host and realm + System.setProperty("java.security.krb5.realm", config.getKerberosRealm()); + System.setProperty("java.security.krb5.kdc", config.getKerberosHost()); + + conf = new Configuration(); + + // enable kerberus + conf.set("hadoop.security.authentication", config.getHadoopAuthentication()); + conf.set("hadoop.security.authorization", config.getHadoopAuthorization()); + + conf.set("fs.defaultFS", hdfsuri); // Set FileSystem URI + conf.set("fs.hdfs.impl", DistributedFileSystem.class.getName()); // Maven stuff? + conf.set("fs.file.impl", LocalFileSystem.class.getName()); // Maven stuff? + + // hack for running locally with fake DNS records + // set this to true if overriding the host name in /etc/hosts + conf.set("dfs.client.use.datanode.hostname", config.getKerberosTestMode()); + + // server principal + // the kerberos principle that the namenode is using + conf.set("dfs.namenode.kerberos.principal.pattern", config.getKerberosPrincipal()); + + // set usergroup stuff + UserGroupInformation.setConfiguration(conf); + UserGroupInformation.loginUserFromKeytab(config.getKerberosKeytabUser(), config.getKerberosKeytabPath()); + + // filesystem for HDFS access is set here + fs = FileSystem.get(conf); + } + } + + public Map hdfsStartOffsets() throws IOException { + Map offsets = new HashMap<>(); + if (topicsRegexString == null) { + topicsRegexString = "^.*$"; // FIXME: all topics if none given + } + + Path workingDir=fs.getWorkingDirectory(); + Path newDirectoryPath= new Path(path); + if(!fs.exists(newDirectoryPath)) { + // Create new Directory + fs.mkdirs(newDirectoryPath); + LOGGER.info("Path {} created.", path); + } + + FileStatus[] directoryStatuses = fs.listStatus(new Path(path), topicFilter); + // Get the directory statuses. Each directory represents a Kafka topic. + if (directoryStatuses.length > 0) { + LOGGER.debug("Found {} matching directories", directoryStatuses.length); + for (FileStatus r : directoryStatuses) { + // Get the file statuses that are inside the directories. + FileStatus[] fileStatuses = fs.listStatus(r.getPath()); + for (FileStatus r2 : fileStatuses) { + String topic = r2.getPath().getParent().getName(); + String[] split = r2.getPath().getName().split("\\."); // The file name can be split to partition parameter and offset parameter. First value is partition and second is offset. + String partition = split[0]; + String offset = split[1]; + TopicPartition topicPartition = new TopicPartition(topic, Integer.parseInt(partition)); + if (!offsets.containsKey(topicPartition)) { + offsets.put(topicPartition, Long.parseLong(offset)+1); + } else { + if (offsets.get(topicPartition) < Long.parseLong(offset)) { + offsets.replace(topicPartition, Long.parseLong(offset)+1); + } + } + } + } + }else { + LOGGER.info("No matching directories found"); + } + return offsets; + } + + private static final PathFilter topicFilter = new PathFilter() { + @Override + public boolean accept(Path path) { + return path.getName().matches(topicsRegexString); // Catches the directory names. + } + }; + + + // try-with-resources handles closing the filesystem automatically. + public void close() { + // FIXME: fs.close() doesn't just affect the current class, it affects all the FileSystem objects that were created using FileSystem.get(URI.create(hdfsuri), conf); in different threads. + if (fs != null) { + try { + fs.close(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + } +} diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java index fc04f930..fa2156cf 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java @@ -22,6 +22,7 @@ import com.teragrep.cfe_39.metrics.topic.TopicCounter; import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -72,6 +73,7 @@ public class KafkaController { private boolean keepRunning; private boolean useMockKafkaConsumer; private final int numOfConsumers; + private Map hdfsStartOffsets; public KafkaController(Config config) { keepRunning = true; @@ -96,6 +98,16 @@ public void run() throws InterruptedException { // register per topic counting List topicCounters = new CopyOnWriteArrayList<>(); + Map topicPartitionStartMap; + // Generate offsets of the already committed records for Kafka + try (HDFSRead hr = new HDFSRead(config) ) { + topicPartitionStartMap = hr.hdfsStartOffsets(); + LOGGER.info("TESTING topicPartitionStartMap"); + LOGGER.info(topicPartitionStartMap.toString()); + } catch (IOException e) { + throw new RuntimeException(e); + } + while (keepRunning) { LOGGER.debug("Scanning for threads"); topicScan(durationStatistics, topicCounters); From 2a3947653050fdf9e878cd9e24578ee33f525b90 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Tue, 14 May 2024 16:21:51 +0300 Subject: [PATCH 059/146] Implemented idempotent kafka consumer functionality using kafka consumer seek(). Altered tests to mirror the changes and added mock HDFS file. --- .../consumers/kafka/KafkaController.java | 12 ++-- .../cfe_39/consumers/kafka/KafkaReader.java | 8 --- .../consumers/kafka/ReadCoordinator.java | 17 +++++- .../com/teragrep/cfe_39/CombinedFullTest.java | 57 +++++++++++++++++- .../com/teragrep/cfe_39/mockHdfsFiles/0.8 | Bin 0 -> 2692 bytes 5 files changed, 78 insertions(+), 16 deletions(-) create mode 100644 src/test/java/com/teragrep/cfe_39/mockHdfsFiles/0.8 diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java index fa2156cf..7a08235b 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java @@ -88,6 +88,7 @@ public KafkaController(Config config) { } else { this.kafkaConsumer = new KafkaConsumer<>(config.getKafkaConsumerProperties(), new ByteArrayDeserializer(), new ByteArrayDeserializer()); } + hdfsStartOffsets = new HashMap<>(); } public void run() throws InterruptedException { @@ -98,12 +99,10 @@ public void run() throws InterruptedException { // register per topic counting List topicCounters = new CopyOnWriteArrayList<>(); - Map topicPartitionStartMap; - // Generate offsets of the already committed records for Kafka + // Generates offsets of the already committed records for Kafka and passes them to the kafka consumers. try (HDFSRead hr = new HDFSRead(config) ) { - topicPartitionStartMap = hr.hdfsStartOffsets(); - LOGGER.info("TESTING topicPartitionStartMap"); - LOGGER.info(topicPartitionStartMap.toString()); + hdfsStartOffsets = hr.hdfsStartOffsets(); + LOGGER.debug("topicPartitionStartMap generated succesfully: {}", hdfsStartOffsets.toString()); } catch (IOException e) { throw new RuntimeException(e); } @@ -162,7 +161,8 @@ private void createReader(String topic, List listPartitionInfo, L ReadCoordinator readCoordinator = new ReadCoordinator( topic, config.getKafkaConsumerProperties(), - output + output, + hdfsStartOffsets ); Thread readThread = new Thread(null, readCoordinator, topic+testi); // Starts the thread with readCoordinator that creates the consumer and subscribes to the topic. threads.add(readThread); diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReader.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReader.java index ed14887f..5526f7ec 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReader.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReader.java @@ -53,14 +53,6 @@ public void read() { ConsumerRecord record = kafkaRecordsIterator.next(); LOGGER.debug("adding from offset: " + record.offset()); recordOffsetObjectList.add(new RecordOffset(record.topic(), record.partition(), record.offset(), record.value())); - -/* // SKIPPING IDEMPOTENT CONSUMER IMPLEMENTATION FOR NOW! - boolean checkStuff = checkIfProcessed(record.topic(), record.partition(), record.offset()); // Create checkIfProcessed method. Checks if the record has already been processed and stored in HDFS. - if (!checkStuff) { - recordOffsetObjectList.add(new RecordOffsetObject(record.topic(), record.partition(), record.offset(), record.value())); - }else{ - // The consumer should update its offsets to effectively mark the message as consumed to ensure it is not redelivered, and no further action takes place. - }*/ } if (!recordOffsetObjectList.isEmpty()) { diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinator.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinator.java index 69f8df0e..ac61a5ea 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinator.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinator.java @@ -18,6 +18,7 @@ package com.teragrep.cfe_39.consumers.kafka; import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -33,15 +34,18 @@ public class ReadCoordinator implements Runnable { private final Properties readerKafkaProperties; private final Consumer> callbackFunction; private boolean run = true; + private final Map hdfsStartOffsets; public ReadCoordinator( String queueTopic, Properties readerKafkaProperties, - Consumer> callbackFunction) + Consumer> callbackFunction, + Map hdfsStartOffsets) { this.queueTopic = queueTopic; this.readerKafkaProperties = readerKafkaProperties; this.callbackFunction = callbackFunction; + this.hdfsStartOffsets = hdfsStartOffsets; } private KafkaReader createKafkaReader(Properties readerKafkaProperties, @@ -62,6 +66,17 @@ private KafkaReader createKafkaReader(Properties readerKafkaProperties, kafkaConsumer.subscribe(Collections.singletonList(topic)); } + Set assignment = kafkaConsumer.assignment(); + // Seek the consumer to topic partition offset defined by the latest record that is committed to HDFS. + for (TopicPartition topicPartition : assignment) { + if (hdfsStartOffsets.containsKey(topicPartition)) { + long position = kafkaConsumer.position(topicPartition); + if (position < hdfsStartOffsets.get(topicPartition)) { + kafkaConsumer.seek(topicPartition, hdfsStartOffsets.get(topicPartition)); + } + } + } + return new KafkaReader(kafkaConsumer, callbackFunction); } diff --git a/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java b/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java index 8d98e056..67783f21 100644 --- a/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java +++ b/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java @@ -23,6 +23,8 @@ import java.sql.Timestamp; import java.util.*; import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import java.util.stream.Stream; public class CombinedFullTest { @@ -65,7 +67,8 @@ public static void teardownMiniCluster() { } @Test - public void kafkaAndAvroFullTest() throws InterruptedException { + public void kafkaAndAvroFullTest() throws InterruptedException, IOException { + insertMockFiles(); // Maybe add modifier that allows adding or not adding the mock file. config.setMaximumFileSize(3000); // 10 loops (140 records) are in use at the moment, and that is sized at 36,102 bytes. KafkaController kafkaController = new KafkaController(config); Thread.sleep(10000); @@ -251,4 +254,56 @@ record = reader.next(record); } fs.close(); } + + private void insertMockFiles() throws IOException { + String path = config.getHdfsPath()+"/"+"testConsumerTopic"; // "hdfs:///opt/teragrep/cfe_39/srv/testConsumerTopic" + String hdfsURI = config.getHdfsuri(); + // ====== Init HDFS File System Object + Configuration conf = new Configuration(); + // Set FileSystem URI + conf.set("fs.defaultFS", hdfsURI); + // Because of Maven + conf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); + conf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); + // Set HADOOP user + System.setProperty("HADOOP_USER_NAME", "hdfs"); + System.setProperty("hadoop.home.dir", "/"); + //Get the filesystem - HDFS + FileSystem fs = FileSystem.get(URI.create(hdfsURI), conf); + + + //==== Create directory if not exists + Path workingDir = fs.getWorkingDirectory(); + // Sets the directory where the data should be stored, if the directory doesn't exist then it's created. + Path newDirectoryPath = new Path(path); + if (!fs.exists(newDirectoryPath)) { + // Create new Directory + fs.mkdirs(newDirectoryPath); + LOGGER.debug("Path {} created.", path); + } + + String dir = System.getProperty("user.dir")+"/src/test/java/com/teragrep/cfe_39/mockHdfsFiles"; + Set listOfFiles = Stream.of(Objects.requireNonNull(new File(dir).listFiles())) + .filter(file -> !file.isDirectory()) + .map(File::getName) + .collect(Collectors.toSet()); + // Loop through all the avro files + for (String fileName : listOfFiles) { + String pathname = dir + "/" + fileName; + File avroFile = new File(pathname); + //==== Write file + LOGGER.debug("Begin Write file into hdfs"); + //Create a path + Path hdfswritepath = new Path(newDirectoryPath + "/" + avroFile.getName()); // filename should be set according to the requirements: 0.12345 where 0 is Kafka partition and 12345 is Kafka offset. + if (fs.exists(hdfswritepath)) { + throw new RuntimeException("File " + avroFile.getName() + " already exists"); + } + Path readPath = new Path(avroFile.getPath()); + // Add conditions if file filtering is required for tests. + fs.copyFromLocalFile(readPath, hdfswritepath); + LOGGER.debug("End Write file into hdfs"); + LOGGER.debug("\nFile committed to HDFS, file writepath should be: {}\n", hdfswritepath.toString()); + } + fs.close(); + } } diff --git a/src/test/java/com/teragrep/cfe_39/mockHdfsFiles/0.8 b/src/test/java/com/teragrep/cfe_39/mockHdfsFiles/0.8 new file mode 100644 index 0000000000000000000000000000000000000000..4272d084652e2c0b730fb21a6c925e7b691a67fa GIT binary patch literal 2692 zcmd5-T}TvB6n0iuR94xg@X7GDMD(yTOyjt#8!Py;;zqi#YCSBA+?~0*&bIU8ompcM zAp*sqUV@(LAw(jAA_xikl^~Fys6gpO5Q1I;*?R7wJL7KYuDfYy_%d_voO92d^L^hr zHylqnZjW1mvbi75>S;9@+4@!2pkSH=Rb~u? zVG<*a5fW&ut&=f>lEVoJrA6KCk4_ch7?O;11Iik)=6{t|m85&x^V@|u=rD=?IOqnJ zw!4TD$2MS)S(F0qgU(@&0z!)4}p!8(FC|9Ai1=oSNiQfIIIaj1k9w3E=k5Ht@()%Xx2-!rXt9qVD{VCs-hG0H~XiO zvaVIXq;fCTZ>?Ho-J_zxNF*F7SC@B}bs>yZd|-vyiVAz$G`Un#*azDpmFr)>`PnM$ zF2(y;FYf~&IxPrOK`EUjY&AOTXc1CQG0zoK%GEK~+%7n> z*SlIz5pHT6n{56Vc(3j*b?+Wi@6L#lkxw0F_A&L^eCoEi{8B|KCs_ zL}U;NR+HFWb+nA^u6+Z(E}wmT#{ID*`{kMkwp?cY%FJ7{$7id#c828{v&9R|%`seC z2Z)-=iIr0hpW;ANQpogG1QZej>}d}$EYSt>+G9N+qa#dg9-xycOa@zsa Date: Wed, 15 May 2024 10:55:57 +0300 Subject: [PATCH 060/146] Fixed bug in topic partition offset map creation in HDFSRead.java. Added another mock hdfs file for testing. --- .../cfe_39/consumers/kafka/HDFSRead.java | 3 +-- .../java/com/teragrep/cfe_39/mockHdfsFiles/0.13 | Bin 0 -> 1982 bytes 2 files changed, 1 insertion(+), 2 deletions(-) create mode 100644 src/test/java/com/teragrep/cfe_39/mockHdfsFiles/0.13 diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSRead.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSRead.java index 8fd83f64..37b0c37f 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSRead.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSRead.java @@ -138,7 +138,7 @@ public Map hdfsStartOffsets() throws IOException { if (!offsets.containsKey(topicPartition)) { offsets.put(topicPartition, Long.parseLong(offset)+1); } else { - if (offsets.get(topicPartition) < Long.parseLong(offset)) { + if (offsets.get(topicPartition) < Long.parseLong(offset)+1) { offsets.replace(topicPartition, Long.parseLong(offset)+1); } } @@ -160,7 +160,6 @@ public boolean accept(Path path) { // try-with-resources handles closing the filesystem automatically. public void close() { - // FIXME: fs.close() doesn't just affect the current class, it affects all the FileSystem objects that were created using FileSystem.get(URI.create(hdfsuri), conf); in different threads. if (fs != null) { try { fs.close(); diff --git a/src/test/java/com/teragrep/cfe_39/mockHdfsFiles/0.13 b/src/test/java/com/teragrep/cfe_39/mockHdfsFiles/0.13 new file mode 100644 index 0000000000000000000000000000000000000000..6bfa102137805bce7b4e0a9ab7d86d82d5a29837 GIT binary patch literal 1982 zcmd6m&ubGw6vu5%ghB-)q10M&T?8-5GTo+%i&bnJNToHHwt6U~jJuP~l--?mX121F zf)}O!_yY=x7B6~HD}o1a-o#T+dJz8tL3;G2uT9i+n`{ku$Z3AO@4T7$yzeVsU+75m z;6=_fVOEHXA01dyW#|*tP&qLfx73X4VRvgi6$*#hrTSgrV{~msMd5vt@i0yV z#|U=g6cr|9T%3A-5G(L8msC>b#bsgTvLJP7qW@)_+O$E2x#jeBZA@knvs-xB_rras z?tf0Nk6l^)w0QK#mnRnrogWe}IyZw`Pj7#G^Ywf3&VGGJLpe=Hx~`xz#$e7E(ltGY zl-X++OPA-A9mQI3@I2DB31;UObY+%$P#Mo8W?IDXYN+1Y_SVj?0 zA^7xoM|VFbHv%XqOvk2XMlR(?ry+hdUmAg-m^!pMTAZ04El%cPS*z=C?Yf(ody0jr z6Dk6SLaERbb3O1Pq=V6H!cR)bYiw_HrB)2o57R8X)*@vbIPyGoIdOag43|Zxvq&?O zU8&;*qZ1vDC?{8+hYhg31Z*3xtJqeWU>j&tY%YP8F!$niuqABi2)1>z@7XSFZxgTT zu-$8pEng}XOaIqy$?o2o+memVR%qb0u3(jL&bWdDi^@1+$6BJ*6QSkf@LUsGYwxQj zYeow?ukC2<*_W-+>W$D!KmK`l7p-@Rb@0~aMwQlsCf*usCt7P&Z~a3%9%;#4eGy*j K!-;QEUcUgt@Q}Iy literal 0 HcmV?d00001 From a1845f4547220ee4ce750aa8bf174eea89688d64 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Thu, 16 May 2024 16:10:14 +0300 Subject: [PATCH 061/146] Changed logging to follow the Java logging standard (issue #12). --- src/main/java/com/teragrep/cfe_39/Config.java | 6 +++--- .../consumers/kafka/DatabaseOutput.java | 21 ++++++------------- .../cfe_39/consumers/kafka/HDFSPrune.java | 8 +++---- .../cfe_39/consumers/kafka/HDFSRead.java | 4 ++-- .../cfe_39/consumers/kafka/HDFSWrite.java | 10 ++++----- .../consumers/kafka/KafkaController.java | 8 +++---- .../cfe_39/consumers/kafka/KafkaReader.java | 2 +- .../consumers/kafka/queue/QueueUtilities.java | 2 +- .../cfe_39/metrics/DurationStatistics.java | 8 +------ 9 files changed, 27 insertions(+), 42 deletions(-) diff --git a/src/main/java/com/teragrep/cfe_39/Config.java b/src/main/java/com/teragrep/cfe_39/Config.java index 6f664291..9e2b93d5 100644 --- a/src/main/java/com/teragrep/cfe_39/Config.java +++ b/src/main/java/com/teragrep/cfe_39/Config.java @@ -57,11 +57,11 @@ public class Config { Config() throws IOException { Properties properties = new Properties(); Path configPath = Paths.get(System.getProperty("cfe_30.config.location", System.getProperty("user.dir") + "/etc/application.properties")); - LOGGER.info("Loading application config '" + configPath.toAbsolutePath() + "'"); + LOGGER.info("Loading application config <[{}]>", configPath.toAbsolutePath()); try(InputStream inputStream = Files.newInputStream(configPath)) { properties.load(inputStream); - LOGGER.debug("Got configuration: " + properties); + LOGGER.debug("Got configuration: <{}>", properties); } // HDFS @@ -103,7 +103,7 @@ public class Config { // Just for loggers to work Path log4j2Config = Paths.get(properties.getProperty("log4j2.configurationFile", System.getProperty("user.dir") + "/etc/log4j2.properties")); - LOGGER.info("Loading log4j2 config from '" + log4j2Config.toRealPath() + "'"); + LOGGER.info("Loading log4j2 config from <[{}]>", log4j2Config.toRealPath()); Configurator.reconfigure(log4j2Config.toUri()); } diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java index ded4cc52..b3f5f112 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java @@ -152,7 +152,7 @@ public void accept(List recordOffsetObjectList) { long thisTime = Instant.now().toEpochMilli(); long ftook = thisTime - lastTimeCalled; topicCounter.setKafkaLatency(ftook); - LOGGER.debug(ANSI_BLUE + "Fuura searching your batch for <[" + table + "]> with records <" + recordOffsetObjectList.size() + "> and took <" + (ftook) + "> milliseconds. <" + (recordOffsetObjectList.size() * 1000L / ftook) + "> EPS. " + ANSI_RESET); + LOGGER.debug("{}Fuura searching your batch for <[{}]> with records <{}> and took <{}> milliseconds. <{}> EPS. {}", ANSI_BLUE, table, recordOffsetObjectList.size(), (ftook), (recordOffsetObjectList.size() * 1000L / ftook), ANSI_RESET); long batchBytes = 0L; // The recordOffsetObjectList loop will go through all the objects in the list. @@ -183,9 +183,9 @@ public void accept(List recordOffsetObjectList) { // Records left to consume in the current partition. boolean b = committedToHdfs(syslogAvroWriter.getFileSize(), (RecordOffset) lastObject); if (b) { - LOGGER.debug("Target file size reached, file {} stored to {} in HDFS", syslogFile.getName(), lastObject.getTopic() + "/" + lastObject.getPartition() + "." + lastObject.getOffset()); + LOGGER.debug("Target file size reached, file <{}> stored to <{}> in HDFS", syslogFile.getName(), lastObject.getTopic() + "/" + lastObject.getPartition() + "." + lastObject.getOffset()); } else { - LOGGER.debug("Target file size not yet reached, continuing writing records to {}.", syslogFile.getName()); + LOGGER.debug("Target file size not yet reached, continuing writing records to <{}>.", syslogFile.getName()); } } else { // Previous partition was fully consumed. Commit file to HDFS and create a new AVRO-file. @@ -261,9 +261,9 @@ public void accept(List recordOffsetObjectList) { // boolean b = committedToHdfs(approximatedSize + capacity, lastObject); // FIXME: approximatedSize is not working properly without the use of flush() after append. File sizes are all over the place. boolean b = committedToHdfs(syslogAvroWriter.getFileSize() + capacity, (RecordOffset) lastObject); if (b) { - LOGGER.debug("Target file size reached, file {} stored to {} in HDFS", syslogFile.getName(), lastObject.getTopic()+"/"+lastObject.getPartition()+"."+lastObject.getOffset()); + LOGGER.debug("Target file size reached, file <{}> stored to <{}> in HDFS", syslogFile.getName(), lastObject.getTopic()+"/"+lastObject.getPartition()+"."+lastObject.getOffset()); }else { - LOGGER.debug("Target file size not yet reached, continuing writing records to {}.", syslogFile.getName()); + LOGGER.debug("Target file size not yet reached, continuing writing records to <{}>.", syslogFile.getName()); } // if more records can be inserted, update epochMicros_last with the timestamp of the last inserted record. epochMicros_last = epochMicros; @@ -311,16 +311,7 @@ public void accept(List recordOffsetObjectList) { topicCounter.addToTotalBytes(batchBytes); topicCounter.addToTotalRecords(recordOffsetObjectList.size()); - LOGGER.debug( - ANSI_GREEN - + "Sent batch for <[" + table + "]> " - + "with records <" + recordOffsetObjectList.size() + "> " - + "and size <" + batchBytes / 1024 + "> KB " - + "took <" + (took) + "> milliseconds. " - + "<" + rps + "> RPS. " - + "<" + bps / 1024 + "> KB/s " - + ANSI_RESET - ); + LOGGER.debug("{}Sent batch for <[{}]> with records <{}> and size <{}> KB took <{}> milliseconds. <{}> RPS. <{}> KB/s {}", ANSI_GREEN, table, recordOffsetObjectList.size(), batchBytes / 1024, (took), rps, bps / 1024, ANSI_RESET); lastTimeCalled = Instant.now().toEpochMilli(); } diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSPrune.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSPrune.java index 4140bde8..2a895862 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSPrune.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSPrune.java @@ -72,7 +72,7 @@ public HDFSPrune(Config config, String topicName) throws IOException { if (!fs.exists(newDirectoryPath)) { // Create new Directory fs.mkdirs(newDirectoryPath); - LOGGER.info("Path {} created.", path); + LOGGER.info("Path <{}> created.", path); } }else { // Code for initializing the class with kerberos. @@ -115,7 +115,7 @@ public HDFSPrune(Config config, String topicName) throws IOException { if (!fs.exists(newDirectoryPath)) { // Create new Directory fs.mkdirs(newDirectoryPath); - LOGGER.info("Path {} created.", path); + LOGGER.info("Path <{}> created.", path); } } long pruneOffset = config.getPruneOffset(); @@ -130,11 +130,11 @@ public void prune() throws IOException { // Delete old files if (a.getModificationTime() < cutOffEpoch) { boolean delete = fs.delete(a.getPath(), true); - LOGGER.info("Deleted file " + a.getPath()); + LOGGER.info("Deleted file <{}>", a.getPath()); } } } else { - LOGGER.info("No files found in directory {}", new Path(newDirectoryPath + "/")); + LOGGER.info("No files found in directory <{}>", new Path(newDirectoryPath + "/")); } } } diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSRead.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSRead.java index 37b0c37f..17ed5330 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSRead.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSRead.java @@ -119,13 +119,13 @@ public Map hdfsStartOffsets() throws IOException { if(!fs.exists(newDirectoryPath)) { // Create new Directory fs.mkdirs(newDirectoryPath); - LOGGER.info("Path {} created.", path); + LOGGER.info("Path <{}> created.", path); } FileStatus[] directoryStatuses = fs.listStatus(new Path(path), topicFilter); // Get the directory statuses. Each directory represents a Kafka topic. if (directoryStatuses.length > 0) { - LOGGER.debug("Found {} matching directories", directoryStatuses.length); + LOGGER.debug("Found <{}> matching directories", directoryStatuses.length); for (FileStatus r : directoryStatuses) { // Get the file statuses that are inside the directories. FileStatus[] fileStatuses = fs.listStatus(r.getPath()); diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java index 05667105..df19f3b4 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java @@ -141,7 +141,7 @@ public void commit(File syslogFile, long lastEpochMicros) { if (!fs.exists(newDirectoryPath)) { // Create new Directory fs.mkdirs(newDirectoryPath); - LOGGER.info("Path {} created.", path); + LOGGER.info("Path <{}> created.", path); } //==== Write file @@ -151,7 +151,7 @@ public void commit(File syslogFile, long lastEpochMicros) { if (fs.exists(hdfswritepath)) { throw new RuntimeException("File " + fileName + " already exists"); } else { - LOGGER.info("Path {} doesn't exist.", path); + LOGGER.info("Path <{}> doesn't exist.", path); } /*//Init output stream @@ -173,7 +173,7 @@ public void commit(File syslogFile, long lastEpochMicros) { // updateTimestamp(hdfswritepath, lastEpochMicros); LOGGER.debug("End Write file into hdfs"); boolean delete = syslogFile.delete(); // deletes the avro-file from the local disk now that it has been committed to HDFS. - LOGGER.info("\nFile committed to HDFS, file writepath should be: {}\n", hdfswritepath.toString()); + LOGGER.info("\nFile committed to HDFS, file writepath should be: <{}>\n", hdfswritepath); } catch (IOException e) { throw new RuntimeException(e); @@ -188,7 +188,7 @@ public void commit(File syslogFile, long lastEpochMicros) { if (!fs.exists(newDirectoryPath)) { // Create new Directory fs.mkdirs(newDirectoryPath); - LOGGER.debug("Path {} created.", path); + LOGGER.debug("Path <{}> created.", path); } //==== Write file @@ -214,7 +214,7 @@ public void commit(File syslogFile, long lastEpochMicros) { outputStream.close(); LOGGER.debug("End Write file into hdfs"); boolean delete = syslogFile.delete(); // deletes the avro-file from the local disk now that it has been committed to HDFS. - LOGGER.debug("\nFile committed to HDFS, file writepath should be: {}\n", hdfswritepath.toString()); + LOGGER.debug("\nFile committed to HDFS, file writepath should be: <{}>\n", hdfswritepath); } catch (IOException e) { throw new RuntimeException(e); diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java index 7a08235b..ed3ab602 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java @@ -102,7 +102,7 @@ public void run() throws InterruptedException { // Generates offsets of the already committed records for Kafka and passes them to the kafka consumers. try (HDFSRead hr = new HDFSRead(config) ) { hdfsStartOffsets = hr.hdfsStartOffsets(); - LOGGER.debug("topicPartitionStartMap generated succesfully: {}", hdfsStartOffsets.toString()); + LOGGER.debug("topicPartitionStartMap generated succesfully: <{}>", hdfsStartOffsets); } catch (IOException e) { throw new RuntimeException(e); } @@ -116,7 +116,7 @@ public void run() throws InterruptedException { long topicScanDelay = 30000L; Thread.sleep(topicScanDelay); for (String topic_name : activeTopics) { - LOGGER.info("topic that is being bruned: " + topic_name); + LOGGER.info("topic that is being bruned: <{}>", topic_name); if (topic_name != null) { try { HDFSPrune hdfsPrune = new HDFSPrune(config, topic_name); @@ -207,13 +207,13 @@ private void topicScan(DurationStatistics durationStatistics, List // Activate all the found in-active topics, in other words create consumer groups for all of them using the createReader()-function. foundPartitions.forEach((k, v) -> { - LOGGER.debug("Activating topic <"+k+">"); + LOGGER.debug("Activating topic <{}>", k); try { createReader(k, v, topicCounters); activeTopics.add(k); durationStatistics.addAndGetThreads(1); } catch (SQLException sqlException) { - LOGGER.error("Topic <"+k+"> not activated due to reader creation error: " + sqlException); + LOGGER.error("Topic <{}> not activated due to reader creation error: ", k, sqlException); } }); durationStatistics.report(); diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReader.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReader.java index 5526f7ec..7e7df627 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReader.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReader.java @@ -51,7 +51,7 @@ public void read() { List recordOffsetObjectList = new ArrayList<>(); while (kafkaRecordsIterator.hasNext()) { ConsumerRecord record = kafkaRecordsIterator.next(); - LOGGER.debug("adding from offset: " + record.offset()); + LOGGER.debug("adding from offset: <{}>", record.offset()); recordOffsetObjectList.add(new RecordOffset(record.topic(), record.partition(), record.offset(), record.value())); } diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/queue/QueueUtilities.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/queue/QueueUtilities.java index 46873bd4..b37056db 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/queue/QueueUtilities.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/queue/QueueUtilities.java @@ -39,7 +39,7 @@ static BiPredicate getFileMatcher(String queueNamePre } else if (!basicFileAttributes.isRegularFile()) { return false; } else { - LOGGER.trace("getFileMatcher returning: " + path); + LOGGER.trace("getFileMatcher returning: <{}>", path); return true; } }; diff --git a/src/main/java/com/teragrep/cfe_39/metrics/DurationStatistics.java b/src/main/java/com/teragrep/cfe_39/metrics/DurationStatistics.java index d33b8b1b..244bcc7b 100644 --- a/src/main/java/com/teragrep/cfe_39/metrics/DurationStatistics.java +++ b/src/main/java/com/teragrep/cfe_39/metrics/DurationStatistics.java @@ -89,13 +89,7 @@ public long getTotalRecords() { } public void log() { - LOGGER.info( - "## Processed records <" + records.getCount() + "> " + - "and size <" + bytes.getCount() / 1024 + "> KB " + - "during <" + samplingIntervalStat.getCount() + "> ms / " + - "Metrics for the preceding minute: <" + recordsPerSecondStat.getOneMinuteRate() + "> RPS. " + - "<" + bytesPerSecondStat.getOneMinuteRate() / 1024 + "> KB/s " - ); + LOGGER.info("## Processed records <{}> and size <{}> KB during <{}> ms / Metrics for the preceding minute: <{}> RPS. <{}> KB/s ", records.getCount(), bytes.getCount() / 1024, samplingIntervalStat.getCount(), recordsPerSecondStat.getOneMinuteRate(), bytesPerSecondStat.getOneMinuteRate() / 1024); samplingIntervalStat.dec(samplingIntervalStat.getCount()); } public long addAndGetThreads(long delta) { From 0aa31194f8d235929b3973f89d579e48a09afe25 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Thu, 16 May 2024 16:15:54 +0300 Subject: [PATCH 062/146] Improved boolean value naming in DatabaseOutput.java. --- .../com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java index b3f5f112..2b5e756d 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java @@ -181,8 +181,8 @@ public void accept(List recordOffsetObjectList) { try { if (lastObject.getTopic().equals(recordOffsetObject.getTopic()) & lastObject.getPartition().equals(recordOffsetObject.getPartition())) { // Records left to consume in the current partition. - boolean b = committedToHdfs(syslogAvroWriter.getFileSize(), (RecordOffset) lastObject); - if (b) { + boolean fileCommitted = committedToHdfs(syslogAvroWriter.getFileSize(), (RecordOffset) lastObject); + if (fileCommitted) { LOGGER.debug("Target file size reached, file <{}> stored to <{}> in HDFS", syslogFile.getName(), lastObject.getTopic() + "/" + lastObject.getPartition() + "." + lastObject.getOffset()); } else { LOGGER.debug("Target file size not yet reached, continuing writing records to <{}>.", syslogFile.getName()); From 1d0ae4862b0f03fd28645987e33e9627e975bf93 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Mon, 27 May 2024 10:50:29 +0300 Subject: [PATCH 063/146] Removed distribution management from pom.xml. --- pom.xml | 6 ------ 1 file changed, 6 deletions(-) diff --git a/pom.xml b/pom.xml index f87cfd4c..8528ce16 100644 --- a/pom.xml +++ b/pom.xml @@ -184,10 +184,4 @@ 2.0.7 - - - scrapyard.xnet.fi-teragrep-releases - https://scrapyard.xnet.fi/repository/teragrep-releases/ - - \ No newline at end of file From 2c50a714cc419b677c324aa99d95c8ca4d14ffc5 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Mon, 27 May 2024 11:05:25 +0300 Subject: [PATCH 064/146] Fixed typo. --- .../com/teragrep/cfe_39/consumers/kafka/KafkaController.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java index ed3ab602..31eade4e 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java @@ -116,7 +116,7 @@ public void run() throws InterruptedException { long topicScanDelay = 30000L; Thread.sleep(topicScanDelay); for (String topic_name : activeTopics) { - LOGGER.info("topic that is being bruned: <{}>", topic_name); + LOGGER.info("topic that is being pruned: <{}>", topic_name); if (topic_name != null) { try { HDFSPrune hdfsPrune = new HDFSPrune(config, topic_name); From 84b3234512a971367695cff7440323a1f971c208 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Mon, 27 May 2024 11:06:59 +0300 Subject: [PATCH 065/146] Fixed boolean name. --- .../com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java index 2b5e756d..2ab5f70b 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java @@ -258,9 +258,9 @@ public void accept(List recordOffsetObjectList) { epochMicros_last = epochMicros; } // Check if there is still room in syslogAvroWriter for another syslogRecord. Commit syslogAvroWriter to HDFS if no room left, emptying it out in the process. - // boolean b = committedToHdfs(approximatedSize + capacity, lastObject); // FIXME: approximatedSize is not working properly without the use of flush() after append. File sizes are all over the place. - boolean b = committedToHdfs(syslogAvroWriter.getFileSize() + capacity, (RecordOffset) lastObject); - if (b) { + // boolean fileCommitted = committedToHdfs(approximatedSize + capacity, lastObject); // FIXME: approximatedSize is not working properly without the use of flush() after append. File sizes are all over the place. + boolean fileCommitted = committedToHdfs(syslogAvroWriter.getFileSize() + capacity, (RecordOffset) lastObject); + if (fileCommitted) { LOGGER.debug("Target file size reached, file <{}> stored to <{}> in HDFS", syslogFile.getName(), lastObject.getTopic()+"/"+lastObject.getPartition()+"."+lastObject.getOffset()); }else { LOGGER.debug("Target file size not yet reached, continuing writing records to <{}>.", syslogFile.getName()); From e2153faac32a584388202fb13d3a43d8db4f2bdc Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Mon, 27 May 2024 15:12:37 +0300 Subject: [PATCH 066/146] Uncommented out code in pom.xml. --- pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index 8528ce16..fff4e00f 100644 --- a/pom.xml +++ b/pom.xml @@ -143,7 +143,7 @@ org.apache.hadoop hadoop-minicluster ${hadoop.version} - + test @@ -163,7 +163,7 @@ org.mockito mockito-core 4.11.0 - + test From ea10fa910e39681601e1a793dc729638932bf8b2 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Tue, 28 May 2024 10:05:54 +0300 Subject: [PATCH 067/146] Changed naming of message to payload in avro schema. Made changes to code and mock files affected by the name change. --- src/main/avro/KafkaRecord.avsc | 4 +-- .../consumers/kafka/DatabaseOutput.java | 2 +- .../com/teragrep/cfe_39/CombinedFullTest.java | 30 +++++++++--------- .../com/teragrep/cfe_39/mockHdfsFiles/0.13 | Bin 1982 -> 1621 bytes .../com/teragrep/cfe_39/mockHdfsFiles/0.8 | Bin 2692 -> 0 bytes .../com/teragrep/cfe_39/mockHdfsFiles/0.9 | Bin 0 -> 2861 bytes 6 files changed, 18 insertions(+), 18 deletions(-) delete mode 100644 src/test/java/com/teragrep/cfe_39/mockHdfsFiles/0.8 create mode 100644 src/test/java/com/teragrep/cfe_39/mockHdfsFiles/0.9 diff --git a/src/main/avro/KafkaRecord.avsc b/src/main/avro/KafkaRecord.avsc index 0004dfcb..2b55cad6 100644 --- a/src/main/avro/KafkaRecord.avsc +++ b/src/main/avro/KafkaRecord.avsc @@ -3,13 +3,13 @@ "name": "SyslogRecord", "fields": [ {"name": "timestamp", "type": "long"}, - {"name": "message", "type": "string"}, {"name": "directory", "type": "string"}, {"name": "stream", "type": "string"}, {"name": "host", "type": "string"}, {"name": "input", "type": "string"}, {"name": "partition", "type": "string"}, {"name": "offset", "type": "long"}, - {"name": "origin", "type": "string"} + {"name": "origin", "type": "string"}, + {"name": "payload", "type": "string"} ] } \ No newline at end of file diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java index 2ab5f70b..57481d79 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java @@ -240,7 +240,7 @@ public void accept(List recordOffsetObjectList) { final long epochMicros = rfc3339ToEpoch(new RFC5424Timestamp(rfc5424Frame.timestamp).toZonedDateTime()); SyslogRecord syslogRecord = SyslogRecord.newBuilder() .setTimestamp(epochMicros) - .setMessage(rfc5424Frame.msg.toString()) + .setPayload(rfc5424Frame.msg.toString()) .setDirectory(rfc5424Frame.structuredData.getValue(teragrepDirectory).toString()) .setStream(rfc5424Frame.structuredData.getValue(teragrepStreamName).toString()) // Or is sourcetype/stream supposed to be rfc5424Frame.appName.toString() instead? .setHost(rfc5424Frame.hostname.toString()) diff --git a/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java b/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java index 67783f21..c883f008 100644 --- a/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java +++ b/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java @@ -184,7 +184,7 @@ public void hdfsReadCheck() throws IOException { // Create the list of files to read from HDFS. Test setup is created so each of the 0-9 partitions will have 2 files with offsets of 8 and 13. List filenameList = new ArrayList<>(); for (int i = 0; i <= 9; i++) { - filenameList.add(i + "." + 8); + filenameList.add(i + "." + 9); filenameList.add(i + "." + 13); } int looper = 0; @@ -205,46 +205,46 @@ record = reader.next(record); LOGGER.info(record.toString()); // Assert records here like it is done in KafkaConsumerTest.avroReader(). if (looper <= 0) { - Assertions.assertEquals("{\"timestamp\": 1650872090804000, \"message\": \"[WARN] 2022-04-25 07:34:50,804 com.teragrep.jla_02.Log4j Log - Log4j warn says hi!\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 0, \"origin\": \"jla-02.default\"}", record.toString()); + Assertions.assertEquals("{\"timestamp\": 1650872090804000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 0, \"origin\": \"jla-02.default\", \"payload\": \"[WARN] 2022-04-25 07:34:50,804 com.teragrep.jla_02.Log4j Log - Log4j warn says hi!\"}", record.toString()); looper++; } else if (looper == 1) { - Assertions.assertEquals("{\"timestamp\": 1650872090806000, \"message\": \"[ERROR] 2022-04-25 07:34:50,806 com.teragrep.jla_02.Log4j Log - Log4j error says hi!\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 1, \"origin\": \"jla-02.default\"}", record.toString()); + Assertions.assertEquals("{\"timestamp\": 1650872090806000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 1, \"origin\": \"jla-02.default\", \"payload\": \"[ERROR] 2022-04-25 07:34:50,806 com.teragrep.jla_02.Log4j Log - Log4j error says hi!\"}", record.toString()); looper++; } else if (looper == 2) { - Assertions.assertEquals("{\"timestamp\": 1650872090822000, \"message\": \"470647 [Thread-3] INFO com.teragrep.jla_02.Logback Daily - Logback-daily says hi.\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 2, \"origin\": \"jla-02\"}", record.toString()); + Assertions.assertEquals("{\"timestamp\": 1650872090822000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 2, \"origin\": \"jla-02\", \"payload\": \"470647 [Thread-3] INFO com.teragrep.jla_02.Logback Daily - Logback-daily says hi.\"}", record.toString()); looper++; } else if (looper == 3) { - Assertions.assertEquals("{\"timestamp\": 1650872090822000, \"message\": \"470646 [Thread-3] INFO com.teragrep.jla_02.Logback Audit - Logback-audit says hi.\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 3, \"origin\": \"jla-02\"}", record.toString()); + Assertions.assertEquals("{\"timestamp\": 1650872090822000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 3, \"origin\": \"jla-02\", \"payload\": \"470646 [Thread-3] INFO com.teragrep.jla_02.Logback Audit - Logback-audit says hi.\"}", record.toString()); looper++; } else if (looper == 4) { - Assertions.assertEquals("{\"timestamp\": 1650872090822000, \"message\": \"470647 [Thread-3] INFO com.teragrep.jla_02.Logback Metric - Logback-metric says hi.\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 4, \"origin\": \"jla-02\"}", record.toString()); + Assertions.assertEquals("{\"timestamp\": 1650872090822000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 4, \"origin\": \"jla-02\", \"payload\": \"470647 [Thread-3] INFO com.teragrep.jla_02.Logback Metric - Logback-metric says hi.\"}", record.toString()); looper++; } else if (looper == 5) { - Assertions.assertEquals("{\"timestamp\": 1650872092238000, \"message\": \"25.04.2022 07:34:52.238 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info audit says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 5, \"origin\": \"jla-02.default\"}", record.toString()); + Assertions.assertEquals("{\"timestamp\": 1650872092238000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 5, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.238 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info audit says hi!]\"}", record.toString()); looper++; } else if (looper == 6) { - Assertions.assertEquals("{\"timestamp\": 1650872092239000, \"message\": \"25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info daily says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 6, \"origin\": \"jla-02.default\"}", record.toString()); + Assertions.assertEquals("{\"timestamp\": 1650872092239000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 6, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info daily says hi!]\"}", record.toString()); looper++; } else if (looper == 7) { - Assertions.assertEquals("{\"timestamp\": 1650872092239000, \"message\": \"25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info metric says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 7, \"origin\": \"jla-02.default\"}", record.toString()); + Assertions.assertEquals("{\"timestamp\": 1650872092239000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 7, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info metric says hi!]\"}", record.toString()); looper++; } else if (looper == 8) { - Assertions.assertEquals("{\"timestamp\": 1650872092240000, \"message\": \"25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn audit says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 8, \"origin\": \"jla-02.default\"}", record.toString()); + Assertions.assertEquals("{\"timestamp\": 1650872092240000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 8, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn audit says hi!]\"}", record.toString()); looper++; } else if (looper == 9) { - Assertions.assertEquals("{\"timestamp\": 1650872092240000, \"message\": \"25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn daily says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 9, \"origin\": \"jla-02.default\"}", record.toString()); + Assertions.assertEquals("{\"timestamp\": 1650872092240000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 9, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn daily says hi!]\"}", record.toString()); looper++; } else if (looper == 10) { - Assertions.assertEquals("{\"timestamp\": 1650872092241000, \"message\": \"25.04.2022 07:34:52.241 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn metric says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 10, \"origin\": \"jla-02.default\"}", record.toString()); + Assertions.assertEquals("{\"timestamp\": 1650872092241000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 10, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.241 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn metric says hi!]\"}", record.toString()); looper++; } else if (looper == 11) { - Assertions.assertEquals("{\"timestamp\": 1650872092241000, \"message\": \"25.04.2022 07:34:52.241 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error audit says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 11, \"origin\": \"jla-02.default\"}", record.toString()); + Assertions.assertEquals("{\"timestamp\": 1650872092241000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 11, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.241 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error audit says hi!]\"}", record.toString()); looper++; } else if (looper == 12) { - Assertions.assertEquals("{\"timestamp\": 1650872092242000, \"message\": \"25.04.2022 07:34:52.242 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error daily says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 12, \"origin\": \"jla-02.default\"}", record.toString()); + Assertions.assertEquals("{\"timestamp\": 1650872092242000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 12, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.242 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error daily says hi!]\"}", record.toString()); looper++; } else { - Assertions.assertEquals("{\"timestamp\": 1650872092243000, \"message\": \"25.04.2022 07:34:52.243 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error metric says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 13, \"origin\": \"jla-02.default\"}", record.toString()); + Assertions.assertEquals("{\"timestamp\": 1650872092243000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 13, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.243 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error metric says hi!]\"}", record.toString()); looper = 0; LOGGER.info("Partition {} passed assertions.", partitionCounter); partitionCounter++; diff --git a/src/test/java/com/teragrep/cfe_39/mockHdfsFiles/0.13 b/src/test/java/com/teragrep/cfe_39/mockHdfsFiles/0.13 index 6bfa102137805bce7b4e0a9ab7d86d82d5a29837..553f5957dc19a8bbd9e4c8091774346054355abd 100644 GIT binary patch delta 348 zcmdnTf0bv#^ohHkPTau7UXWOslb@I}xtCF0F6901bE=chy97MC>~6Fre+SbFmUT=i z7dEz@+I;sN>*PNyHrzZ`Oa>xWMhqWvJ0=&ficeNzu3+3U`67$(Ta01eB9;wIDIYE%%maH+3T$2v*n?ZydbL9!4$Mt0 QDauSnm>A@}=K;Ej09^Ey82|tP delta 550 zcmcc0vyXqmboSiT;^M^g)QJmZCQg4kc|N14f_nFLkqs_^Gp?0M|F}Kh-h=4`%UY%n zr45T(Pi?;Yj_8W`o| zr>CbDsg$G^mso)XI4A3~Xj*t!F&PNS00ndnjPz1c(-KQ_N_=OmsKjFHI-so=HbQLe z0NN@w*_f#mVyB)sc1w6cc9?_g5J9tJL$cIz3_G^5tVgnAvOV)su^FjFMfpVviKQu- zB?vD~wr8~g8!LupY(ZkB4Gv=$9E3O>XspEKDwbZD(P*IsG8!Bz5@<$?%=_PoVe~GR z4L}!vxBxM_2NV~R=d$){FMv2ZH?^cFGa2D-ETJR`3L*_q5J{m~s5IN>HM)fWA&S;X diff --git a/src/test/java/com/teragrep/cfe_39/mockHdfsFiles/0.8 b/src/test/java/com/teragrep/cfe_39/mockHdfsFiles/0.8 deleted file mode 100644 index 4272d084652e2c0b730fb21a6c925e7b691a67fa..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 2692 zcmd5-T}TvB6n0iuR94xg@X7GDMD(yTOyjt#8!Py;;zqi#YCSBA+?~0*&bIU8ompcM zAp*sqUV@(LAw(jAA_xikl^~Fys6gpO5Q1I;*?R7wJL7KYuDfYy_%d_voO92d^L^hr zHylqnZjW1mvbi75>S;9@+4@!2pkSH=Rb~u? zVG<*a5fW&ut&=f>lEVoJrA6KCk4_ch7?O;11Iik)=6{t|m85&x^V@|u=rD=?IOqnJ zw!4TD$2MS)S(F0qgU(@&0z!)4}p!8(FC|9Ai1=oSNiQfIIIaj1k9w3E=k5Ht@()%Xx2-!rXt9qVD{VCs-hG0H~XiO zvaVIXq;fCTZ>?Ho-J_zxNF*F7SC@B}bs>yZd|-vyiVAz$G`Un#*azDpmFr)>`PnM$ zF2(y;FYf~&IxPrOK`EUjY&AOTXc1CQG0zoK%GEK~+%7n> z*SlIz5pHT6n{56Vc(3j*b?+Wi@6L#lkxw0F_A&L^eCoEi{8B|KCs_ zL}U;NR+HFWb+nA^u6+Z(E}wmT#{ID*`{kMkwp?cY%FJ7{$7id#c828{v&9R|%`seC z2Z)-=iIr0hpW;ANQpogG1QZej>}d}$EYSt>+G9N+qa#dg9-xycOa@zsaNDm?iiq57vC#Ti(}1GAB4_Te2nVTCpg4G-{%CVRC;)$8t+`cb$jUg0Jw#$3(qnuakSn0=D;b}HSOX?ZnqWls zRDJwXo5%iwdgzdrFE5saowC{6rXArxC=?78f^Yf}yrZIxup^9hyaxY>d3`+({wZ~b zz+YPWZFn~k-s!63R`yOg_-3|=Zw4UTKSKO1(G3yM-Mc>s3Q-S3@i^#$vXUgSZ*}`= zDK$>3&U~IZ_IzW(E7pTI{UmsMLXxdCI82Q*;PcA+L>~AOb+>eiZ??e)Z~U6#J%}_U zan7m$lK5Xcb-bq&YP4}DwV2IDoqsd(%(|)c5yFdSDYtuYj@+DJeg7@ xclt_c%;YX+CY}Z9L$l!F_@_%Qb^hj*+^)%M+c#FFn*ULyoRT}|^E0_BeFd&-vA_TT literal 0 HcmV?d00001 From 96da5842e150a5d59bfa6c71f96fc090196117c5 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Tue, 28 May 2024 10:09:40 +0300 Subject: [PATCH 068/146] Removed unused test file and related dependency. --- pom.xml | 5 - .../java/com/teragrep/cfe_39/PruneTest.java | 130 ------------------ 2 files changed, 135 deletions(-) delete mode 100644 src/test/java/com/teragrep/cfe_39/PruneTest.java diff --git a/pom.xml b/pom.xml index fff4e00f..d8d08cc1 100644 --- a/pom.xml +++ b/pom.xml @@ -127,11 +127,6 @@ avro 1.11.3
- - org.apache.avro - avro-mapred - 1.11.3 - org.apache.hadoop diff --git a/src/test/java/com/teragrep/cfe_39/PruneTest.java b/src/test/java/com/teragrep/cfe_39/PruneTest.java deleted file mode 100644 index b322fa8f..00000000 --- a/src/test/java/com/teragrep/cfe_39/PruneTest.java +++ /dev/null @@ -1,130 +0,0 @@ -package com.teragrep.cfe_39; - -import com.teragrep.cfe_39.avro.SyslogRecord; -import org.apache.avro.Schema; -import org.apache.avro.mapred.AvroKey; -import org.apache.avro.mapred.AvroValue; -import org.apache.avro.mapreduce.AvroJob; -import org.apache.avro.mapreduce.AvroKeyInputFormat; -import org.apache.avro.mapreduce.AvroKeyValueOutputFormat; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.conf.Configured; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.IntWritable; -import org.apache.hadoop.io.LongWritable; -import org.apache.hadoop.io.NullWritable; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.mapreduce.InputSplit; -import org.apache.hadoop.mapreduce.Job; -import org.apache.hadoop.mapreduce.Mapper; -import org.apache.hadoop.mapreduce.Reducer; -import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; -import org.apache.hadoop.mapreduce.lib.input.FileSplit; -import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat; -import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; -import org.apache.hadoop.util.Tool; -import org.apache.hadoop.util.ToolRunner; - -import java.io.IOException; - -// Shelved for now. Using HDFS modification timestamps instead of avro-mapred for pruning the files. - -// This class should be compiled into a jar-file that is then sent to the hadoop cluster for running the job when needed. Maven should be configured to do the jar-packaging etc. -/* -The records are stored inside files that are 64MB in size and named depending on which Kafka partition offset the last stored record belongs to. -In other words the files are inside topic_name-directory and there are at least one file per partition, depending on the load size of records that are fetched from Kafka topics. - -The AVRO-files that hold the records can house over 240,000 records each (at 64MB), but that is just an estimate because the record sizes vary widely. -This means that the partition offsets are not guaranteed to be same for all the files in a topic. In other words the same number of records could be distributed between 3 files on one partition and 2 files on another partition because of the different individual record sizes. -Lets take 2 topic partitions and their files as an example: - -topic_name/0.25 (contains partition 0 records between offsets 0 and 25) -topic_name/0.50 (contains partition 0 records between offsets 26 and 50) -topic_name/1.35 (contains partition 1 records between offsets 0 and 35) -topic_name/1.55 (contains partition 1 records between offsets 36 and 55) - -Timestamps for the record offsets are linear: record 0.1 is timed first, record 1.1 is second, 0.2 is third, 1.2 is fourth, etc. -Lets say that pruning cutoff epoch lands between records 0.30 and 1.30. This means that the file topic_name/0.25 is pruned but topic_name/1.35 is not. -topic_name/1.35 will contain records from between offsets 0 and 35. Only the records at or above offset 30 are coherent, everything under that can be considered to be garbage data if they are ever queried. - -Should the file be altered so the garbage data can be removed from the file? Or should the records only be filtered out when responding to queries with result sets? -The filtering method is most likely the least resource intensive method as it can be done during the MapReduce, and the amount of garbage record shouldn't be too much. If the main function decides on the cutoff epoch, then it can also track it for the datasource function to use for filtering. -But in any case the pruning should include deleting AVRO-files that hold only outdated records that should be pruned. The handling of the leftover garbage records can be handled later in the MapReduce of the datasource component queries. -*/ - -// The main function that will call for pruning will know the topic name (aka. directory path). The pruning will be done in directory basis, aka. in topic basis, so tracking the topic name is not important for the MapReduce as the input path already contains the topic name. -// Instead, the partition and offset values together with timestamp are important for pruning. The MapReduce function should create a list of key-value pairs where key is the partition+offset and value is the timestamp, where timestamp is smaller than the cutoff_epoch defined by input arguments. -// The pruning of old records can be called in KafkaController.java row 112, using the activeTopics list as a input argument for topic names. This way the records are pruned every time new ones are added. Make sure there are no concurrency issues with the HDFS writer. Most likely there is a need for pruning-controller class that will manage the directory/topic scanning etc. -public class PruneTest extends Configured implements Tool { - static long cutoff_epoch; - // TimestampMapper takes a SyslogRecord as input and outputs a key-value pair of record partition+"."+offset and timestamp of the record. - public static class TimestampMapper extends Mapper, NullWritable, Text, LongWritable> { - @Override - public void map(AvroKey key, NullWritable value, Context context) throws IOException, InterruptedException { - - // TODO: Add context.getInputSplit(); functionality to the mapper that allows tracking of the filenames that the records originate from. - // FIXME: Casting context.getInputSplit()).getPath() to FileSplit shouldn't work anymore with newer versions of hadoop. Input split class now returns TaggedInputSplit instead. - String filename = ((FileSplit) context.getInputSplit()).getPath().getName(); - // If the FileSplit casting starts working for some reason on the newest version of MapReduce, it can be used to filter out only the specific filenames that should be deleted from HDFS. This way the size of the MapReduce output will be optimized and doesn't need any additional processing by the client. - - long timestamp = key.datum().getTimestamp(); - CharSequence partition = key.datum().getPartition(); - long offset = key.datum().getOffset(); - context.write(new Text(partition + "." + offset), new LongWritable(timestamp)); // Changed the output so the key will contain both the partition and offset (partition + "." + offset), while value will contain the timestamp. This way reduce can do the pruning with the mapper output. - } - } - - - // Removes all the key-timestamp pairs that have timestamp over the cutoff_epoch. What is left are list of keys (outdated records) that should be removed from the HDFS database. - public static class TimestampReducer extends Reducer, AvroValue> { - @Override - public void reduce(Text key, Iterable values, Context context) throws IOException, InterruptedException { - long timestamp; - for (LongWritable value : values) { - timestamp = value.get(); - if (timestamp < cutoff_epoch) { // TODO: inclusive or exclusive? Exclusive for now. - context.write(new AvroKey(key.toString()), new AvroValue(timestamp)); - } - } - } - } - - public int run(String[] args) throws Exception { - if (args.length != 3) { - System.err.println("Usage: MapReduceTimestampPrune "); - return -1; - } - try { - cutoff_epoch = Long.parseLong(args[2]); - } catch (NumberFormatException e) { - System.err.println("Usage: 3rd input argument should be parseable to long"); - return -1; - } - - Configuration conf = new Configuration(); - Job job = Job.getInstance(conf, "timestamp prune"); - job.setJarByClass(PruneTest.class); - - SequenceFileInputFormat.setInputPaths(job, new Path(args[0])); // The input path should be the directory where the AVRO-files are held. setInputPaths can take either directory or file as input, not sure if using directory has the same effect as having a list of files. - FileOutputFormat.setOutputPath(job, new Path(args[1])); // Output path is where the results of the MapReduce are stored. - - job.setInputFormatClass(AvroKeyInputFormat.class); - job.setMapperClass(TimestampMapper.class); - AvroJob.setInputKeySchema(job, SyslogRecord.getClassSchema()); - job.setMapOutputKeyClass(Text.class); - job.setMapOutputValueClass(LongWritable.class); - - job.setOutputFormatClass(AvroKeyValueOutputFormat.class); - job.setReducerClass(TimestampReducer.class); - AvroJob.setOutputKeySchema(job, Schema.create(Schema.Type.STRING)); - AvroJob.setOutputValueSchema(job, Schema.create(Schema.Type.LONG)); - - return (job.waitForCompletion(true) ? 0 : 1); - } - - // Set input directory to be the topic directory. - public static void main(String[] args) throws Exception { - int res = ToolRunner.run(new PruneTest(), args); // arg1 is and arg2 is , output path should be a new HDFS directory that does not exist and input path should be the HDFS directory with AVRO-files that we have generated in tests. - System.exit(res); - } -} From 022832c8b423efc7466b7aead56cc317a45a0002 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Tue, 28 May 2024 15:58:39 +0300 Subject: [PATCH 069/146] Cleaned up commenting. Removed duplicate code in HDFSWrite.java. --- src/main/java/com/teragrep/cfe_39/Config.java | 3 +- .../consumers/kafka/DatabaseOutput.java | 71 +++----- .../cfe_39/consumers/kafka/HDFSPrune.java | 8 +- .../cfe_39/consumers/kafka/HDFSRead.java | 26 +-- .../cfe_39/consumers/kafka/HDFSWrite.java | 168 ++++-------------- .../consumers/kafka/KafkaController.java | 53 +----- .../cfe_39/consumers/kafka/KafkaReader.java | 12 +- .../kafka/MockKafkaConsumerFactoryTemp.java | 8 - .../consumers/kafka/ReadCoordinator.java | 4 +- .../consumers/kafka/SyslogAvroWriter.java | 7 +- .../com/teragrep/cfe_39/CombinedFullTest.java | 31 ++-- .../java/com/teragrep/cfe_39/HdfsTest.java | 9 +- 12 files changed, 103 insertions(+), 297 deletions(-) diff --git a/src/main/java/com/teragrep/cfe_39/Config.java b/src/main/java/com/teragrep/cfe_39/Config.java index 9e2b93d5..3492b5b9 100644 --- a/src/main/java/com/teragrep/cfe_39/Config.java +++ b/src/main/java/com/teragrep/cfe_39/Config.java @@ -32,7 +32,6 @@ import java.util.Properties; public class Config { - // kafka private final String queueTopicPattern; private final Properties kafkaConsumerProperties; private static final Logger LOGGER = LoggerFactory.getLogger(Config.class); @@ -52,7 +51,7 @@ public class Config { private final int numOfConsumers; private final long pruneOffset; - // TODO: Set up configuration check for important parameters. Remove old unused parameters. + // TODO: Set up configuration check for important parameters. Config() throws IOException { Properties properties = new Properties(); diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java index 57481d79..31c26d48 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java @@ -35,10 +35,9 @@ import java.nio.ByteBuffer; -// The kafka stream should first be deserialized using rlo_06 and then serialized again using avro and stored in HDFS. -// The target where the record is stored in HDFS is based on the topic, partition and offset. ie. topic_name/0.123456 where offset is 123456 - -// The mock consumer is activated for testing using the configuration file: readerKafkaProperties.getProperty("useMockKafkaConsumer", "false") +/* The kafka stream should first be deserialized using rlo_06 and then serialized again using avro and stored in HDFS. + The target where the record is stored in HDFS is based on the topic, partition and offset. ie. topic_name/0.123456 where offset is 123456 + The mock consumer is activated for testing using the configuration file: readerKafkaProperties.getProperty("useMockKafkaConsumer", "false")*/ public class DatabaseOutput implements Consumer> { private static final Logger LOGGER = LoggerFactory.getLogger(DatabaseOutput.class); @@ -56,7 +55,6 @@ public class DatabaseOutput implements Consumer> { public static final String ANSI_GREEN = "\u001B[32m"; public static final String ANSI_BLUE = "\u001B[34m"; private SyslogAvroWriter syslogAvroWriter; - // private final long minimumFreeSpace; // TODO: Check if needs to be implemented to the code. private final long maximumFileSize; private final WritableQueue writableQueue; private final ByteBuffer sourceConcatenationBuffer; @@ -71,7 +69,6 @@ public class DatabaseOutput implements Consumer> { private final SDVector originHostname; private File syslogFile; private final Config config; - // private long approximatedSize; // FIXME: Not working properly when flush() is not used after append in the AVRO-file. private long epochMicros_last; DatabaseOutput( @@ -84,8 +81,7 @@ public class DatabaseOutput implements Consumer> { this.table = table; this.durationStatistics = durationStatistics; this.topicCounter = topicCounter; -// this.minimumFreeSpace = 32000000; // TODO: CHECK RIGHT VALUE FOR minimumFreeSpace - this.maximumFileSize = config.getMaximumFileSize(); // Maximum file size should be 64M (64000000). 60800000 is 95% of 64M which should be a good approximation point. + this.maximumFileSize = config.getMaximumFileSize(); // queueDirectory and queueNamePrefix are only used for temporarily storing the AVRO-serialized files before committing them to HDFS when the file size reaches the threshold (or all records are processed). this.writableQueue = new WritableQueue( @@ -102,7 +98,6 @@ public class DatabaseOutput implements Consumer> { this.eventNodeSourceHostname = new SDVector("event_node_source@48577","hostname"); this.eventNodeRelayHostname = new SDVector("event_node_relay@48577","hostname"); this.originHostname = new SDVector("origin@48577","hostname"); -// this.approximatedSize = 0; this.epochMicros_last = 0L; } @@ -123,7 +118,6 @@ boolean committedToHdfs(long fileSize, RecordOffset recordOffsetObject) { writableQueue.setQueueNamePrefix(recordOffsetObject.getTopic()+recordOffsetObject.getPartition()); syslogFile = writableQueue.getNextWritableFile(); syslogAvroWriter = new SyslogAvroWriter(syslogFile); -// approximatedSize = syslogAvroWriter.getFileSize(); // resets the size approximation. return true; } } catch (IOException ioException) { @@ -142,11 +136,11 @@ private long rfc3339ToEpoch(ZonedDateTime zonedDateTime) { return Math.addExact(sec, instant.getNano() / NANOS_PER_MICROS); } - // Input parameter is a list of RecordOffsetObjects. Each object contains a record and its metadata (topic, partition and offset). - // Each partition will get their set of exclusive AVRO-files in HDFS. - // The target where the record is stored in HDFS is based on the topic, partition and last offset. ie. topic_name/0.123456 where last written record's offset is 123456. - // AVRO-file with a path/name that starts with topic_name/0.X should only contain records from the 0th partition of topic named topic_name, topic_name/1.X should only contain records from 1st partition, etc. - // AVRO-files are created dynamically, thus it is not known which record (and its offset) is written to the file last before committing it to HDFS. The final name for the HDFS file is decided only when the file is committed to HDFS. + /* Input parameter is a list of RecordOffsetObjects. Each object contains a record and its metadata (topic, partition and offset). + Each partition will get their set of exclusive AVRO-files in HDFS. + The target where the record is stored in HDFS is based on the topic, partition and last offset. ie. topic_name/0.123456 where last written record's offset is 123456. + AVRO-file with a path/name that starts with topic_name/0.X should only contain records from the 0th partition of topic named topic_name, topic_name/1.X should only contain records from 1st partition, etc. + AVRO-files are created dynamically, thus it is not known which record (and its offset) is written to the file last before committing it to HDFS. The final name for the HDFS file is decided only when the file is committed to HDFS.*/ @Override public void accept(List recordOffsetObjectList) { long thisTime = Instant.now().toEpochMilli(); @@ -155,11 +149,11 @@ public void accept(List recordOffsetObjectList) { LOGGER.debug("{}Fuura searching your batch for <[{}]> with records <{}> and took <{}> milliseconds. <{}> EPS. {}", ANSI_BLUE, table, recordOffsetObjectList.size(), (ftook), (recordOffsetObjectList.size() * 1000L / ftook), ANSI_RESET); long batchBytes = 0L; - // The recordOffsetObjectList loop will go through all the objects in the list. - // While it goes through the list, the contents of the objects are serialized into an AVRO-file. - // When the file size is about to go above 64M, commit the file into HDFS using the latest topic/partition/offset values as the filename and start fresh with a new empty AVRO-file. - // Serialize the object that was going to make the file go above 64M into the now empty AVRO-file and continue the loop. - // TODO: If the prod-environment recordOffsetObjectList ordering is different from what it is in the test environment, add a function that reorders the list based on partition and offset (or better yet, make several AVRO-files that are being used at the same time rather than doing it one AVRO-file at a time as the offset ordering within partitions should always be correct in all scenarios). + /* The recordOffsetObjectList loop will go through all the objects in the list. + While it goes through the list, the contents of the objects are serialized into an AVRO-file. + When the file size is about to go above 64M, commit the file into HDFS using the latest topic/partition/offset values as the filename and start fresh with a new empty AVRO-file. + Serialize the object that was going to make the file go above 64M into the now empty AVRO-file and continue the loop. + TODO: If the prod-environment recordOffsetObjectList ordering is different from what it is in the test environment, add a function that reorders the list based on partition and offset (or better yet, make several AVRO-files that are being used at the same time rather than doing it one AVRO-file at a time as the offset ordering within partitions should always be correct in all scenarios).*/ Offset lastObject = new NullOffset(); // Abstract OffsetObject class, set to null object before initializing as RecordOffsetObject. long start = Instant.now().toEpochMilli(); // Starts measuring performance here. Measures how long it takes to process the whole recordOffsetObjectList. // This loop goes through all the records of the mock data in a single session. @@ -172,7 +166,6 @@ public void accept(List recordOffsetObjectList) { writableQueue.getNextWritableFile(); // The HDFS filename is only finalized when the AVRO-serialized file is finalized, because every Kafka-record added to the file is going to change the offset that is going to be used for the filename. syslogAvroWriter = new SyslogAvroWriter(syslogFile); -// approximatedSize = syslogAvroWriter.getFileSize(); // resets the size approximation. lastObject = recordOffsetObject; } catch (IOException ioException) { throw new IllegalArgumentException(ioException); @@ -197,7 +190,6 @@ public void accept(List recordOffsetObjectList) { writableQueue.setQueueNamePrefix(recordOffsetObject.getTopic() + recordOffsetObject.getPartition()); syslogFile = writableQueue.getNextWritableFile(); syslogAvroWriter = new SyslogAvroWriter(syslogFile); -// approximatedSize = syslogAvroWriter.getFileSize(); // resets the size approximation. } } catch (IOException ioException) { throw new UncheckedIOException(ioException); @@ -210,25 +202,10 @@ public void accept(List recordOffsetObjectList) { rfc5424Frame.load(inputStream); try { if(rfc5424Frame.next()) { - // rfc5424Frame has loaded the record data, it's ready for deserialization. - // Implement AVRO serialization for the Kafka records here, preparing the data for writing to HDFS. - // Write all the data into a file using AVRO. - // The size of each AVRO-serialized file should be as close to 64M as possible. - - /* - EXAMPLE FORMAT FROM PTH_06.KafkaReconrdConverter WHICH SHOULD BE USED FOR AVRO-FORMAT! - return RowFactory.create( - Timestamp.from(instant), // 0 "_time", DataTypes.TimestampType - UTF8String.fromBytes(message).toString(), // 1 "_raw", DataTypes.StringType - UTF8String.fromBytes(index).toString(), // 2 "directory", DataTypes.StringType - UTF8String.fromBytes(sourcetype).toString(),// 3 "stream", DataTypes.StringType - UTF8String.fromBytes(hostname).toString(), // 4 "host", DataTypes.StringType, - UTF8String.fromBytes(input).toString(), // 5 "input", DataTypes.StringType - partition, // 6 "partition", DataTypes.StringType - offset, // 7 "offset", DataTypes.LongType - UTF8String.fromBytes(origin).toString() // 8 "origin", DataTypes.StringType - ); - */ +/* rfc5424Frame has loaded the record data, it's ready for deserialization. + Implement AVRO serialization for the Kafka records here, preparing the data for writing to HDFS. + Write all the data into a file using AVRO. + The size of each AVRO-serialized file should be as close to 64M as possible.*/ // input final byte[] source = eventToSource(); @@ -258,7 +235,6 @@ public void accept(List recordOffsetObjectList) { epochMicros_last = epochMicros; } // Check if there is still room in syslogAvroWriter for another syslogRecord. Commit syslogAvroWriter to HDFS if no room left, emptying it out in the process. - // boolean fileCommitted = committedToHdfs(approximatedSize + capacity, lastObject); // FIXME: approximatedSize is not working properly without the use of flush() after append. File sizes are all over the place. boolean fileCommitted = committedToHdfs(syslogAvroWriter.getFileSize() + capacity, (RecordOffset) lastObject); if (fileCommitted) { LOGGER.debug("Target file size reached, file <{}> stored to <{}> in HDFS", syslogFile.getName(), lastObject.getTopic()+"/"+lastObject.getPartition()+"."+lastObject.getOffset()); @@ -269,9 +245,7 @@ public void accept(List recordOffsetObjectList) { epochMicros_last = epochMicros; // Add syslogRecord to syslogAvroWriter which has room for new syslogRecord. syslogAvroWriter.write(syslogRecord); -// approximatedSize += capacity; lastObject = recordOffsetObject; - // The difference between actual and approximate file size is about 2,4 % with 64M files. So setting the MaximumFileSize to 95 % of the target should make things work. } } catch (IOException e) { throw new RuntimeException(e); @@ -330,10 +304,10 @@ private byte[] eventToOrigin() { } private byte[] eventToSource() { - //input is produced from SD element event_node_source@48577 by - // concatenating "source_module:hostname:source". in case - //if event_node_source@48577 is not available use event_node_relay@48577. - //If neither are present, use null value. + /*input is produced from SD element event_node_source@48577 by + concatenating "source_module:hostname:source". in case + if event_node_source@48577 is not available use event_node_relay@48577. + If neither are present, use null value.*/ sourceConcatenationBuffer.clear(); @@ -378,7 +352,6 @@ private byte[] eventToSource() { } - // source_module:hostname:source" sourceConcatenationBuffer.put(source_module); sourceConcatenationBuffer.put((byte) ':'); sourceConcatenationBuffer.put(source_hostname); diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSPrune.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSPrune.java index 2a895862..9bba0e50 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSPrune.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSPrune.java @@ -94,12 +94,12 @@ public HDFSPrune(Config config, String topicName) throws IOException { conf.set("fs.hdfs.impl", DistributedFileSystem.class.getName()); // Maven stuff? conf.set("fs.file.impl", LocalFileSystem.class.getName()); // Maven stuff? - // hack for running locally with fake DNS records - // set this to true if overriding the host name in /etc/hosts + /* hack for running locally with fake DNS records + set this to true if overriding the host name in /etc/hosts*/ conf.set("dfs.client.use.datanode.hostname", config.getKerberosTestMode()); - // server principal - // the kerberos principle that the namenode is using + /* server principal + the kerberos principle that the namenode is using*/ conf.set("dfs.namenode.kerberos.principal.pattern", config.getKerberosPrincipal()); // set usergroup stuff diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSRead.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSRead.java index 17ed5330..d00e17f2 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSRead.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSRead.java @@ -16,9 +16,9 @@ import java.util.Properties; public final class HDFSRead implements AutoCloseable { - // Maps out the latest offset for all the topic partitions available in HDFS. - // The offset map can then be used for kafka consumer seek() method, which will add the idempotent functionality to the consumer. - // Also, because this class should be called outside the loops that generate the consumer groups it should be lightweight to run. + /* Maps out the latest offset for all the topic partitions available in HDFS. + The offset map can then be used for kafka consumer seek() method, which will add the idempotent functionality to the consumer. + Also, because this class should be called outside the loops that generate the consumer groups it should be lightweight to run.*/ private static final Logger LOGGER = LoggerFactory.getLogger(HDFSRead.class); private final FileSystem fs; @@ -29,7 +29,7 @@ public final class HDFSRead implements AutoCloseable { private final String path; public HDFSRead(Config config) throws IOException { - // Check for testmode from config. + // Check if mock kafka consumer is enabled in the config. Properties readerKafkaProperties = config.getKafkaConsumerProperties(); this.useMockKafkaConsumer = Boolean.parseBoolean( readerKafkaProperties.getProperty("useMockKafkaConsumer", "false") @@ -57,26 +57,12 @@ public HDFSRead(Config config) throws IOException { throw new RuntimeException(e); } - /*//==== Create directory if not exists - Path workingDir=fs.getWorkingDirectory(); - Path newDirectoryPath= new Path(path); - if(!fs.exists(newDirectoryPath)) { - // Create new Directory - fs.mkdirs(newDirectoryPath); - LOGGER.info("Path {} created.", path); - }*/ - }else { // Code for initializing the class with kerberos. hdfsuri = config.getHdfsuri(); // Get from config.' path = config.getHdfsPath(); - - // Set HADOOP user here, Kerberus parameters most likely needs to be added here too. - // System.setProperty("HADOOP_USER_NAME", "hdfs"); // Not needed because user authentication is done by kerberos? - // System.setProperty("hadoop.home.dir", "/"); // Not needed because user authentication is done by kerberos? - // set kerberos host and realm System.setProperty("java.security.krb5.realm", config.getKerberosRealm()); System.setProperty("java.security.krb5.kdc", config.getKerberosHost()); @@ -91,8 +77,8 @@ public HDFSRead(Config config) throws IOException { conf.set("fs.hdfs.impl", DistributedFileSystem.class.getName()); // Maven stuff? conf.set("fs.file.impl", LocalFileSystem.class.getName()); // Maven stuff? - // hack for running locally with fake DNS records - // set this to true if overriding the host name in /etc/hosts + /* hack for running locally with fake DNS records + set this to true if overriding the host name in /etc/hosts*/ conf.set("dfs.client.use.datanode.hostname", config.getKerberosTestMode()); // server principal diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java index df19f3b4..9a49f3ed 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java @@ -38,28 +38,24 @@ public class HDFSWrite implements AutoCloseable{ private final String fileName; private final String path; private final FileSystem fs; - private final boolean useMockKafkaConsumer; // test-mode switch + private final boolean useMockKafkaConsumer; // Defines if mock HDFS database is used for testing private final Configuration conf; private final String hdfsuri; - // Create files as whole but stream the contents into them. Avro files 'flush' must be called as few times as possible. Check memory usage impact - // Later make sure to check the avro file flush issue where the file size is all over the place if flush is not used after every append to the file. - public HDFSWrite(Config config, RecordOffset lastObject) throws IOException { - // Check for testmode from config. Properties readerKafkaProperties = config.getKafkaConsumerProperties(); this.useMockKafkaConsumer = Boolean.parseBoolean( readerKafkaProperties.getProperty("useMockKafkaConsumer", "false") ); if (useMockKafkaConsumer) { - // Code for initializing the class in test mode without kerberos. - hdfsuri = config.getHdfsuri(); // Get from config. + // Code for initializing the class for mock hdfs database usage without kerberos. + hdfsuri = config.getHdfsuri(); - // The filepath should be something like hdfs:///opt/teragrep/cfe_39/srv/topic_name/0.12345 where 12345 is offset and 0 the partition. - // In other words the directory named topic_name holds files that are named and arranged based on partition and the partition's offset. Every partition has its own set of unique offset values. - // These values should be fetched from config and other input parameters (topic+partition+offset). + /* The filepath should be something like hdfs:///opt/teragrep/cfe_39/srv/topic_name/0.12345 where 12345 is offset and 0 the partition. + In other words the directory named topic_name holds files that are named and arranged based on partition and the partition's offset. Every partition has its own set of unique offset values. + These values should be fetched from config and other input parameters (topic+partition+offset).*/ path = config.getHdfsPath()+"/"+lastObject.topic; fileName = lastObject.partition+"."+lastObject.offset; // filename should be constructed from partition and offset. @@ -70,7 +66,7 @@ public HDFSWrite(Config config, RecordOffset lastObject) throws IOException { // Because of Maven conf.set("fs.hdfs.impl", DistributedFileSystem.class.getName()); conf.set("fs.file.impl", LocalFileSystem.class.getName()); - // Set HADOOP user here, Kerberus parameters most likely needs to be added here too. + // Set HADOOP user here. System.setProperty("HADOOP_USER_NAME", "hdfs"); System.setProperty("hadoop.home.dir", "/"); // filesystem for HDFS access is set here @@ -82,19 +78,11 @@ public HDFSWrite(Config config, RecordOffset lastObject) throws IOException { }else { - // Code for initializing the class with kerberos. - hdfsuri = config.getHdfsuri(); // Get from config. - - // The filepath should be something like hdfs:///opt/teragrep/cfe_39/srv/topic_name/0.12345 where 12345 is offset and 0 the partition. - // In other words the directory named topic_name holds files that are named and arranged based on partition and the partition's offset. Every partition has its own set of unique offset values. - // The values are fetched from config and input parameters (topic+partition+offset). - path = config.getHdfsPath() + "/" + lastObject.topic; // directory path is constructed from HdfsPath and topic name. - fileName = lastObject.partition + "." + lastObject.offset; // filename should be constructed from partition and offset. - + // Code for initializing the class for kerberized HDFS database usage. + hdfsuri = config.getHdfsuri(); - // Set HADOOP user here, Kerberus parameters most likely needs to be added here too. - // System.setProperty("HADOOP_USER_NAME", "hdfs"); // Not needed because user authentication is done by kerberos? - // System.setProperty("hadoop.home.dir", "/"); // Not needed because user authentication is done by kerberos? + path = config.getHdfsPath() + "/" + lastObject.topic; + fileName = lastObject.partition + "." + lastObject.offset; // set kerberos host and realm System.setProperty("java.security.krb5.realm", config.getKerberosRealm()); @@ -110,12 +98,10 @@ public HDFSWrite(Config config, RecordOffset lastObject) throws IOException { conf.set("fs.hdfs.impl", DistributedFileSystem.class.getName()); // Maven stuff? conf.set("fs.file.impl", LocalFileSystem.class.getName()); // Maven stuff? - // hack for running locally with fake DNS records - // set this to true if overriding the host name in /etc/hosts + // hack for running locally with fake DNS records, set this to true if overriding the host name in /etc/hosts conf.set("dfs.client.use.datanode.hostname", config.getKerberosTestMode()); - // server principal - // the kerberos principle that the namenode is using + // server principal, the kerberos principle that the namenode is using conf.set("dfs.namenode.kerberos.principal.pattern", config.getKerberosPrincipal()); // set usergroup stuff @@ -130,105 +116,33 @@ public HDFSWrite(Config config, RecordOffset lastObject) throws IOException { // Method for committing the AVRO-file to HDFS public void commit(File syslogFile, long lastEpochMicros) { // The code for writing the file to HDFS should be same for both test (non-kerberized access) and prod (kerberized access). - if (useMockKafkaConsumer) { - // CODE FOR TEST-MODE GOES HERE! - //Get the filesystem - HDFS - try { - //==== Create directory if not exists - Path workingDir = fs.getWorkingDirectory(); - // Sets the directory where the data should be stored, if the directory doesn't exist then it's created. - Path newDirectoryPath = new Path(path); - if (!fs.exists(newDirectoryPath)) { - // Create new Directory - fs.mkdirs(newDirectoryPath); - LOGGER.info("Path <{}> created.", path); - } - - //==== Write file - LOGGER.debug("Begin Write file into hdfs"); - //Create a path - Path hdfswritepath = new Path(newDirectoryPath.toString() + "/" + fileName); // filename should be set according to the requirements: 0.12345 where 0 is Kafka partition and 12345 is Kafka offset. - if (fs.exists(hdfswritepath)) { - throw new RuntimeException("File " + fileName + " already exists"); - } else { - LOGGER.info("Path <{}> doesn't exist.", path); - } - - /*//Init output stream - FSDataOutputStream outputStream = fs.create(hdfswritepath); - // Write the file contents of syslogFile to hdfswritepath in HDFS. - // file to bytes[] - - *//*byte[] bytearray = new byte[(int) syslogFile.length()]; - try (FileInputStream inputStream = new FileInputStream(syslogFile)) { - inputStream.read(bytearray); - }*//* - byte[] bytes = Files.readAllBytes(Paths.get(syslogFile.getPath())); // if readAllBytes is not efficient use FileInputStream - outputStream.write(bytes); - outputStream.close();*/ - - Path path = new Path(syslogFile.getPath()); - fs.copyFromLocalFile(path, hdfswritepath); - // fs.setTimes(hdfswritepath, lastEpochMicros, -1); // where mtime is modification time and atime is access time. -1 as input parameter leaves the original atime/mtime value as is. - // updateTimestamp(hdfswritepath, lastEpochMicros); - LOGGER.debug("End Write file into hdfs"); - boolean delete = syslogFile.delete(); // deletes the avro-file from the local disk now that it has been committed to HDFS. - LOGGER.info("\nFile committed to HDFS, file writepath should be: <{}>\n", hdfswritepath); - - } catch (IOException e) { - throw new RuntimeException(e); + try { + //==== Create directory if not exists + Path workingDir = fs.getWorkingDirectory(); + // Sets the directory where the data should be stored, if the directory doesn't exist then it's created. + Path newDirectoryPath = new Path(path); + if (!fs.exists(newDirectoryPath)) { + // Create new Directory + fs.mkdirs(newDirectoryPath); + LOGGER.info("Path <{}> created.", path); } - }else { - //Get the filesystem - HDFS - try { - //==== Create directory if not exists - Path workingDir = fs.getWorkingDirectory(); - // Sets the directory where the data should be stored, if the directory doesn't exist then it's created. - Path newDirectoryPath = new Path(path); - if (!fs.exists(newDirectoryPath)) { - // Create new Directory - fs.mkdirs(newDirectoryPath); - LOGGER.debug("Path <{}> created.", path); - } - - //==== Write file - LOGGER.debug("Begin Write file into hdfs"); - //Create a path - Path hdfswritepath = new Path(newDirectoryPath + "/" + fileName); // filename should be set according to the requirements: 0.12345 where 0 is Kafka partition and 12345 is Kafka offset. - if (fs.exists(hdfswritepath)) { - throw new RuntimeException("File " + fileName + " already exists"); - } - - //Init output stream - FSDataOutputStream outputStream = fs.create(hdfswritepath); - // Write the file contents of syslogFile to hdfswritepath in HDFS. - // file to bytes[] - - /*byte[] bytearray = new byte[(int) syslogFile.length()]; - try (FileInputStream inputStream = new FileInputStream(syslogFile)) { - inputStream.read(bytearray); - }*/ - byte[] bytes = Files.readAllBytes(Paths.get(syslogFile.getPath())); - outputStream.write(bytes); - - outputStream.close(); - LOGGER.debug("End Write file into hdfs"); - boolean delete = syslogFile.delete(); // deletes the avro-file from the local disk now that it has been committed to HDFS. - LOGGER.debug("\nFile committed to HDFS, file writepath should be: <{}>\n", hdfswritepath); - } catch (IOException e) { - throw new RuntimeException(e); + //==== Write file + LOGGER.debug("Begin Write file into hdfs"); + //Create a path + Path hdfswritepath = new Path(newDirectoryPath.toString() + "/" + fileName); // filename should be set according to the requirements: 0.12345 where 0 is Kafka partition and 12345 is Kafka offset. + if (fs.exists(hdfswritepath)) { + throw new RuntimeException("File " + fileName + " already exists"); + } else { + LOGGER.info("Path <{}> doesn't exist.", path); } - } - } - private void updateTimestamp(Path hdfswritepath, long lastEpochMicros) { - // Testing timestamp editing. The new timestamp should be the timestamp of the last record that was added to the AVRO-file. - try { - FileSystem fs_temp = FileSystem.get(URI.create(hdfsuri), conf); - FSDataOutputStream fsDataOutputStream = fs_temp.create(hdfswritepath); - fs_temp.setTimes(hdfswritepath, lastEpochMicros, -1); // where mtime is modification time and atime is access time. -1 as input parameter leaves the original atime/mtime value as is. - fsDataOutputStream.close(); + Path path = new Path(syslogFile.getPath()); + fs.copyFromLocalFile(path, hdfswritepath); + LOGGER.debug("End Write file into hdfs"); + boolean delete = syslogFile.delete(); // deletes the avro-file from the local disk now that it has been committed to HDFS. + LOGGER.info("\nFile committed to HDFS, file writepath should be: <{}>\n", hdfswritepath); + } catch (IOException e) { throw new RuntimeException(e); } @@ -236,14 +150,8 @@ private void updateTimestamp(Path hdfswritepath, long lastEpochMicros) { // try-with-resources handles closing the filesystem automatically. public void close() { - // FIXME: fs.close() doesn't just affect the current class, it affects all the FileSystem objects that were created using FileSystem.get(URI.create(hdfsuri), conf); in different threads. - /*if (fs != null) { - try { - fs.close(); - } catch (IOException e) { - throw new RuntimeException(e); - } - }*/ + /* NoOp + When used here fs.close() doesn't just affect the current class, it affects all the FileSystem objects that were created using FileSystem.get(URI.create(hdfsuri), conf); in different threads.*/ } diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java index 31eade4e..c75c9c03 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java @@ -37,32 +37,8 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; + public class KafkaController { - // ReadCoordinator alone won't allow access to the kafka offsets, it must be done in KafkaReader that is used on rlo_09. - // ReadCoordinator uses the KafkaReader, but it's set as private in rlo_09 and there are no functions for accessing it through ReadCoordinator. - // The enable.auto.commit=false is set in config and it is fetched by the config.getKafkaConsumerProperties(). - // cfe_30 is already using enable.auto.commit=false, so looking through cfe_30 and rlo_09 ReadCoordinator and KafkaReader functions should get the coding on right track. - - // A consumer may opt to commit offsets by itself (enable.auto.commit=false). - // Depending on when it chooses to commit offsets, there are delivery semantics available to the consumer. - // Exactly once: - // - For Kafka topic to External System workflows, to effectively achieve exactly once, you must use an idempotent consumer. - - // An Idempotent Consumer pattern uses a Kafka consumer that can consume the same message any number of times, but only process it once. - // To implement the Idempotent Consumer pattern the recommended approach is to add a table to the database to track processed messages. - // Each message needs to have a unique messageId assigned by the producing service, either within the payload, or as a Kafka message header. - // When a new message is consumed the table is checked for the existence of this message Id. If present, then the message is a duplicate. - // The consumer updates its offsets to effectively mark the message as consumed to ensure it is not redelivered, and no further action takes place. - // If the message Id is not present in the table then a database transaction is started and the message Id is inserted. - // The message is then processed performing the required business logic. Upon completion the transaction is committed. - - // requirements: SKIPPING IDEMPOTENT IMPLEMENTATION FOR NOW! - // 1. The KafkaReader-class must be able to pass the offset values alongside the consumed message to the main class that is calling KafkaReader. - // 2. The main class must handle the consuming of the kafka topics in an idempotent way as stated above. - // This is achieved by using the HDFS filenames to store the topic_name and offset values of Kafka topics. - // In other words Kafka consumers will consume topics normally according to the offsets that Kafka stores internally, - // but the processor of the consumed messages will check the offsets of the messages that are consumed by kafka and - // processes ONLY those messages that have not already been processed based on the offset values stored in HDFS filenames. private static final Logger LOGGER = LoggerFactory.getLogger(KafkaController.class); private final Config config; @@ -143,21 +119,16 @@ private void createReader(String topic, List listPartitionInfo, L // Add the new topicCounter object to the list. topicCounters.add(topicCounter); - // Every consumer is run in a separate thread. - // Consumer group is also handled here, and each consumer of the group runs on separate thread. + /* Every consumer is run in a separate thread. + Consumer group is also handled here, and each consumer of the group runs on separate thread.*/ int numOfThreads = Math.min(numOfConsumers, listPartitionInfo.size()); // Makes sure that there aren't more consumers than available partitions in the consumer group. for (int testi = 1; numOfThreads >= testi; testi++) { - // DatabaseOutput handles transferring the consumed data to storage (S3, mariadb, HDFS, etc.) - // Kafka offset tracking must be included here. - // Topic is figured out in topicScan so the offsets for the topic should be figured out here. Consumer> output = new DatabaseOutput( config, // Configuration settings topic, // String, the name of the topic durationStatistics, // RuntimeStatistics object from metrics topicCounter // TopicCounter object from metrics ); - // The kafka offsets must be passed to HDFS. The consumer must also be set to manual commits so the HDFS can handle managing the commit offsets within the HDFS filenames. - // plain rlo_09.ReadCoordinator won't give access to offset values. Implementing custom rlo_09 code in the package to achieve offset access. ReadCoordinator readCoordinator = new ReadCoordinator( topic, config.getKafkaConsumerProperties(), @@ -172,34 +143,24 @@ private void createReader(String topic, List listPartitionInfo, L } private void topicScan(DurationStatistics durationStatistics, List topicCounters) { - Map> listTopics = kafkaConsumer.listTopics(Duration.ofSeconds(60)); // Topics can be fetched from mock consumer if the consumer has been updated separately with the partition info. - Pattern topicsRegex = Pattern.compile(config.getQueueTopicPattern()); // Mock consumer has the partitions in this format: queueTopicPattern=^testConsumerTopic-*$ - // Find the topics available in Kafka based on given QueueTopicPattern, both active and in-active. - // Need to allow using consumer groups for partition read assignments. aka. load balancing + Map> listTopics = kafkaConsumer.listTopics(Duration.ofSeconds(60)); + Pattern topicsRegex = Pattern.compile(config.getQueueTopicPattern()); +// Find the topics available in Kafka based on given QueueTopicPattern, both active and in-active. Set foundTopics = new HashSet<>(); - - // 1. Add functionality so the partition information is also fetched for the queried topics. At the moment only the topic names are fetched. Map> foundPartitions = new HashMap<>(); - - for (Map.Entry> entry : listTopics.entrySet()) { Matcher matcher = topicsRegex.matcher(entry.getKey()); if (matcher.matches()) { foundTopics.add(entry.getKey()); - - // 2. Add functionality so the partition information is also fetched for the queried topics. foundPartitions.put(entry.getKey(), entry.getValue()); - } } - if (foundTopics.isEmpty()) { throw new IllegalStateException("Pattern <[" + config.getQueueTopicPattern() + "]> found no topics." ); } - // subtract currently active topics from found topics foundTopics.removeAll(activeTopics); - // 3. Subtract currently active partitions from found partitions + // Subtract currently active partitions from found partitions for (String topic_name : activeTopics) { foundPartitions.remove(topic_name); // removes the partitions from the list based on the topic name. } diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReader.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReader.java index 7e7df627..31b100e6 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReader.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReader.java @@ -41,7 +41,7 @@ public void read() { long offset; if (!kafkaRecordsIterator.hasNext()) { // still need to consume more, infinitely loop because connection problems may cause return of an empty iterator - ConsumerRecords kafkaRecords = kafkaConsumer.poll(Duration.ofSeconds(60)); // TODO parametrize + ConsumerRecords kafkaRecords = kafkaConsumer.poll(Duration.ofSeconds(60)); if (kafkaRecords.isEmpty()) { LOGGER.debug("kafkaRecords empty after poll."); } @@ -56,19 +56,15 @@ public void read() { } if (!recordOffsetObjectList.isEmpty()) { - // This is the DatabaseOutput.accept() function. - // Offset and other required data for HDFS storage are added to the input parameters of the accept() function which processes the consumed record. + /* This is the DatabaseOutput.accept() function. + Offset and other required data for HDFS storage are added to the input parameters of the accept() function which processes the consumed record.*/ callbackFunction.accept(recordOffsetObjectList); kafkaConsumer.commitSync(); - /* - commitSync() only commits the offsets that were actually polled and processed. If some offsets were not included in the last poll, then those offsets will not be committed. - It will not commit the latest positions for all subscribed partitions. This would interfere with the Consumer Offset management concept of Kafka to be able to re-start an application where it left off. - * */ } } @Override public void close() { - kafkaConsumer.close(Duration.ofSeconds(60)); // TODO parametrize + kafkaConsumer.close(Duration.ofSeconds(60)); } } diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java index 8c511f58..30153a1e 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java @@ -164,7 +164,6 @@ public static Consumer getConsumer(int threadnum) { endOffsets.put(topicPartition, 14L); mockPartitionInfo.add(new PartitionInfo("testConsumerTopic", i, null, null, null)); } - // consumer.subscribe(Collections.singletonList("testConsumerTopic")); // subscribe if (threadnum == 1) { List oddTopicPartitions = new ArrayList<>(); @@ -201,15 +200,8 @@ public static Consumer getConsumer(int threadnum) { consumer.updateBeginningOffsets(beginningOffsets); - //insert stuff - // consumer.rebalance(topicPartitions); // needed for subscribe - /*for (TopicPartition a : topicPartitions) { - generateEvents(consumer, a.topic(), a.partition()); - }*/ - consumer.updateEndOffsets(endOffsets); consumer.updatePartitions("testConsumerTopic", mockPartitionInfo); return consumer; - // The code for starting consumers in separate threads is located in KafkaController.java line 138. } } \ No newline at end of file diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinator.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinator.java index ac61a5ea..d053d6b0 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinator.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinator.java @@ -54,14 +54,14 @@ private KafkaReader createKafkaReader(Properties readerKafkaProperties, boolean useMockKafkaConsumer) { org.apache.kafka.clients.consumer.Consumer kafkaConsumer; - if (useMockKafkaConsumer) { // Test mode is on, create mock consumers with assigned partitions that are not overlapping with each other. + if (useMockKafkaConsumer) { // Mock kafka consumer is enabled, create mock consumers with assigned partitions that are not overlapping with each other. String name = Thread.currentThread().getName(); // Use thread name to identify which thread is running the code. if (Objects.equals(name, "testConsumerTopic1")) { kafkaConsumer = MockKafkaConsumerFactoryTemp.getConsumer(1); // creates a Kafka MockConsumer that has the odd numbered partitions assigned to it. }else { kafkaConsumer = MockKafkaConsumerFactoryTemp.getConsumer(2); // creates a Kafka MockConsumer that has the even numbered partitions assigned to it. } - } else { // Test mode is off, subscribe method should handle assigning the partitions automatically to the consumer based on group id parameters of readerKafkaProperties. + } else { // Mock kafka consumer is disabled, subscribe method should handle assigning the partitions automatically to the consumer based on group id parameters of readerKafkaProperties. kafkaConsumer = new KafkaConsumer<>(readerKafkaProperties, new ByteArrayDeserializer(), new ByteArrayDeserializer()); kafkaConsumer.subscribe(Collections.singletonList(topic)); } diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java index 4afa932b..604374f8 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java @@ -48,7 +48,6 @@ class SyslogAvroWriter implements AutoCloseable { syncableFileOutputStream = new SyncableFileOutputStream(syslogFile); - // LOGGER.debug("debugging syslogFile, path is: " + syslogFile.getPath()); syncableFileOutputStream.getChannel().tryLock(); if (syslogFile.length() == 0) { @@ -68,10 +67,8 @@ class SyslogAvroWriter implements AutoCloseable { void write(SyslogRecord syslogRecord) throws IOException{ dataFileWriter.append(syslogRecord); - dataFileWriter.flush(); // FIXME: getFileSize() doesn't work properly if dataFileWriter.flush() is not called after appending a new record to the AVRO-file. - - // Avro files 'flush' must be called as few times as possible. Check memory usage impact. Use only automatic flush which is triggered when .close() is called. - // To use the automatic flush AND have a working getFileSize(), the file size must be tracked separately. Approximate the file size by adding the original file size before any appending to the sum of record sizes. + dataFileWriter.flush(); + // getFileSize() doesn't work properly if dataFileWriter.flush() is not called after appending a new record to the AVRO-file. } public void close() throws IOException { diff --git a/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java b/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java index c883f008..0d44ffb5 100644 --- a/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java +++ b/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java @@ -54,7 +54,6 @@ public static void startMiniCluster() throws IOException, InterruptedException { MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf); hdfsCluster = builder.build(); String hdfsURI = "hdfs://localhost:"+ hdfsCluster.getNameNodePort() + "/"; - // System.out.println("hdfsURI: " + hdfsURI); config.setHdfsuri(hdfsURI); DistributedFileSystem fileSystem = hdfsCluster.getFileSystem(); } @@ -73,9 +72,9 @@ public void kafkaAndAvroFullTest() throws InterruptedException, IOException { KafkaController kafkaController = new KafkaController(config); Thread.sleep(10000); kafkaController.run(); - // The avro files should be committed to HDFS now. Check the committed files for any errors. - // There should be 20 files, 10 partitions with each having 2 files assigned to them. - // hdfsReadCheck(); does not work properly if pruning is enabled and prune offset is set too low, which causes the records to be pruned from the database. + /* The avro files should be committed to HDFS now. Check the committed files for any errors. + There should be 20 files, 10 partitions with each having 2 files assigned to them. + hdfsReadCheck(); does not work properly if pruning is enabled and prune offset is set too low, which causes the records to be pruned from the database.*/ if (config.getPruneOffset() == 157784760000L) { try { hdfsReadCheck(); @@ -113,17 +112,17 @@ public void hdfsPruneTest() throws IOException { LOGGER.info("Path {} created.", path); } - // Use either HDFS-file modification timestamps or avro-mapred for pruning. + /* Use either HDFS-file modification timestamps or avro-mapred for pruning. - // The records are in this AVRO format: - // {"timestamp": 1650872092240000, "message": "25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn audit says hi!]", "directory": "jla02logger", "stream": "test:jla02logger:0", "host": "jla-02.default", "input": "imrelp:cfe-06-0.cfe-06.default:", "partition": "8", "offset": 8, "origin": "jla-02.default"} - // Query handler must be implemented in a way that the AVRO files are first opened, then processed to syslog format and then sent to the query requester. The records are processed/filtered based on the given query conditions using MapReduce to make the code capable of processing the vast amounts of records that are expected. - // MapReduce functionalities of the Hadoop cluster: https://hadoop.apache.org/docs/stable/hadoop-mapreduce-client/hadoop-mapreduce-client-core/MapReduceTutorial.html - // Avro side of documentations for MapReduce: https://avro.apache.org/docs/1.11.1/mapreduce-guide/ + The records are in this AVRO format: + {"timestamp": 1650872092240000, "message": "25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn audit says hi!]", "directory": "jla02logger", "stream": "test:jla02logger:0", "host": "jla-02.default", "input": "imrelp:cfe-06-0.cfe-06.default:", "partition": "8", "offset": 8, "origin": "jla-02.default"} + Query handler must be implemented in a way that the AVRO files are first opened, then processed to syslog format and then sent to the query requester. The records are processed/filtered based on the given query conditions using MapReduce to make the code capable of processing the vast amounts of records that are expected. + MapReduce functionalities of the Hadoop cluster: https://hadoop.apache.org/docs/stable/hadoop-mapreduce-client/hadoop-mapreduce-client-core/MapReduceTutorial.html + Avro side of documentations for MapReduce: https://avro.apache.org/docs/1.11.1/mapreduce-guide/ - // Another method for pruning aside using avro-mapred is to use modification timestamp of the avro-file stored in HDFS: - // fs.setTimes(new Path(path+"/"+0.8), Long.parseUnsignedLong("1675930598000"), -1); - // where mtime is modification time and atime is access time. -1 as input parameter leaves the original atime/mtime value as is. + Another method for pruning aside using avro-mapred is to use modification timestamp of the avro-file stored in HDFS: + fs.setTimes(new Path(path+"/"+0.8), Long.parseUnsignedLong("1675930598000"), -1); + where mtime is modification time and atime is access time. -1 as input parameter leaves the original atime/mtime value as is.*/ FileStatus[] fileStatuses = fs.listStatus(new Path(newDirectoryPath + "/")); long count = Arrays.stream(fileStatuses).count(); if (count != 0) { @@ -178,10 +177,10 @@ public void hdfsReadCheck() throws IOException { LOGGER.info("Path {} created.", path); } - // This is the HDFS write path for the files: - // Path hdfswritepath = new Path(newDirectoryPath + "/" + fileName); where newDirectoryPath is config.getHdfsPath() + "/" + lastObject.topic; and filename is lastObject.partition+"."+lastObject.offset; + /* This is the HDFS write path for the files: + Path hdfswritepath = new Path(newDirectoryPath + "/" + fileName); where newDirectoryPath is config.getHdfsPath() + "/" + lastObject.topic; and filename is lastObject.partition+"."+lastObject.offset; - // Create the list of files to read from HDFS. Test setup is created so each of the 0-9 partitions will have 2 files with offsets of 8 and 13. + Create the list of files to read from HDFS. Test setup is created so each of the 0-9 partitions will have 2 files with offsets of 8 and 13.*/ List filenameList = new ArrayList<>(); for (int i = 0; i <= 9; i++) { filenameList.add(i + "." + 9); diff --git a/src/test/java/com/teragrep/cfe_39/HdfsTest.java b/src/test/java/com/teragrep/cfe_39/HdfsTest.java index a067c7cb..9de328eb 100644 --- a/src/test/java/com/teragrep/cfe_39/HdfsTest.java +++ b/src/test/java/com/teragrep/cfe_39/HdfsTest.java @@ -184,12 +184,8 @@ public void hdfsReadCheck(String testConsumerTopic, int partition, long offset) // logger.info("Path "+path+" created."); } - // This is the HDFS write path for the files: - // Path hdfswritepath = new Path(newDirectoryPath + "/" + fileName); where newDirectoryPath is config.getHdfsPath() + "/" + lastObject.topic; and filename is lastObject.partition+"."+lastObject.offset; - - //==== Read files - // logger.info("Read file into hdfs"); - //Create a path + /*==== Read files + Create a path*/ Path hdfsreadpath = new Path(newDirectoryPath + "/" + fileName); // The path should be the same that was used in writing the file to HDFS. //Init input stream FSDataInputStream inputStream = fs.open(hdfsreadpath); @@ -253,7 +249,6 @@ record = reader.next(record); looper = 0; } } - // logger.info(out); inputStream.close(); fs.close(); } From a1fda67179df7e304fab6dc81ca8a270e68d3b8b Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Wed, 29 May 2024 08:14:41 +0300 Subject: [PATCH 070/146] Removed ANSI coloring from logging and fixed wrong log level usage. --- .../teragrep/cfe_39/consumers/kafka/DatabaseOutput.java | 7 ++----- src/test/java/com/teragrep/cfe_39/CombinedFullTest.java | 2 +- 2 files changed, 3 insertions(+), 6 deletions(-) diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java index 31c26d48..99b9914a 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java @@ -51,9 +51,6 @@ public class DatabaseOutput implements Consumer> { private long lastTimeCalled = Instant.now().toEpochMilli(); - public static final String ANSI_RESET = "\u001B[0m"; - public static final String ANSI_GREEN = "\u001B[32m"; - public static final String ANSI_BLUE = "\u001B[34m"; private SyslogAvroWriter syslogAvroWriter; private final long maximumFileSize; private final WritableQueue writableQueue; @@ -146,7 +143,7 @@ public void accept(List recordOffsetObjectList) { long thisTime = Instant.now().toEpochMilli(); long ftook = thisTime - lastTimeCalled; topicCounter.setKafkaLatency(ftook); - LOGGER.debug("{}Fuura searching your batch for <[{}]> with records <{}> and took <{}> milliseconds. <{}> EPS. {}", ANSI_BLUE, table, recordOffsetObjectList.size(), (ftook), (recordOffsetObjectList.size() * 1000L / ftook), ANSI_RESET); + LOGGER.debug("Fuura searching your batch for <[{}]> with records <{}> and took <{}> milliseconds. <{}> EPS. ", table, recordOffsetObjectList.size(), (ftook), (recordOffsetObjectList.size() * 1000L / ftook)); long batchBytes = 0L; /* The recordOffsetObjectList loop will go through all the objects in the list. @@ -285,7 +282,7 @@ public void accept(List recordOffsetObjectList) { topicCounter.addToTotalBytes(batchBytes); topicCounter.addToTotalRecords(recordOffsetObjectList.size()); - LOGGER.debug("{}Sent batch for <[{}]> with records <{}> and size <{}> KB took <{}> milliseconds. <{}> RPS. <{}> KB/s {}", ANSI_GREEN, table, recordOffsetObjectList.size(), batchBytes / 1024, (took), rps, bps / 1024, ANSI_RESET); + LOGGER.debug("Sent batch for <[{}]> with records <{}> and size <{}> KB took <{}> milliseconds. <{}> RPS. <{}> KB/s ", table, recordOffsetObjectList.size(), batchBytes / 1024, (took), rps, bps / 1024); lastTimeCalled = Instant.now().toEpochMilli(); } diff --git a/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java b/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java index 0d44ffb5..30e850c3 100644 --- a/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java +++ b/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java @@ -201,7 +201,7 @@ public void hdfsReadCheck() throws IOException { LOGGER.info("\nReading records from file {}:", hdfsreadpath.toString()); while (reader.hasNext()) { record = reader.next(record); - LOGGER.info(record.toString()); + LOGGER.debug(record.toString()); // Assert records here like it is done in KafkaConsumerTest.avroReader(). if (looper <= 0) { Assertions.assertEquals("{\"timestamp\": 1650872090804000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 0, \"origin\": \"jla-02.default\", \"payload\": \"[WARN] 2022-04-25 07:34:50,804 com.teragrep.jla_02.Log4j Log - Log4j warn says hi!\"}", record.toString()); From 8721d828a5a4f2630949a6c8225e6be472ac536b Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Wed, 29 May 2024 09:09:39 +0300 Subject: [PATCH 071/146] Fixed missing and wrong licenses. --- src/main/java/com/teragrep/cfe_39/Config.java | 57 +++++++++++++----- .../consumers/kafka/DatabaseOutput.java | 57 +++++++++++++----- .../cfe_39/consumers/kafka/HDFSPrune.java | 57 +++++++++++++----- .../cfe_39/consumers/kafka/HDFSRead.java | 46 +++++++++++++++ .../cfe_39/consumers/kafka/HDFSWrite.java | 57 +++++++++++++----- .../consumers/kafka/KafkaController.java | 57 +++++++++++++----- .../cfe_39/consumers/kafka/KafkaReader.java | 57 +++++++++++++----- .../kafka/MockKafkaConsumerFactoryTemp.java | 57 +++++++++++++----- .../cfe_39/consumers/kafka/NullOffset.java | 58 ++++++++++++++----- .../cfe_39/consumers/kafka/Offset.java | 58 ++++++++++++++----- .../consumers/kafka/ReadCoordinator.java | 57 +++++++++++++----- .../cfe_39/consumers/kafka/RecordOffset.java | 57 +++++++++++++----- .../consumers/kafka/SyslogAvroWriter.java | 57 +++++++++++++----- .../consumers/kafka/queue/QueueUtilities.java | 57 +++++++++++++----- .../consumers/kafka/queue/WritableQueue.java | 57 +++++++++++++----- .../cfe_39/metrics/DurationStatistics.java | 57 +++++++++++++----- .../cfe_39/metrics/topic/TopicCounter.java | 57 +++++++++++++----- .../com/teragrep/cfe_39/CombinedFullTest.java | 46 +++++++++++++++ .../java/com/teragrep/cfe_39/HdfsTest.java | 46 +++++++++++++++ .../teragrep/cfe_39/KafkaConsumerTest.java | 46 +++++++++++++++ 20 files changed, 874 insertions(+), 224 deletions(-) diff --git a/src/main/java/com/teragrep/cfe_39/Config.java b/src/main/java/com/teragrep/cfe_39/Config.java index 3492b5b9..c187b200 100644 --- a/src/main/java/com/teragrep/cfe_39/Config.java +++ b/src/main/java/com/teragrep/cfe_39/Config.java @@ -1,18 +1,47 @@ /* - HDFS Data Ingestion for PTH_06 use CFE-39 - Copyright (C) 2022 Fail-Safe IT Solutions Oy - - 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. + * HDFS Data Ingestion for PTH_06 use CFE-39 + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. */ package com.teragrep.cfe_39; diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java index 99b9914a..28475126 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java @@ -1,18 +1,47 @@ /* - HDFS Data Ingestion for PTH_06 use CFE-39 - Copyright (C) 2022 Fail-Safe IT Solutions Oy - - 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. + * HDFS Data Ingestion for PTH_06 use CFE-39 + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. */ package com.teragrep.cfe_39.consumers.kafka; diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSPrune.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSPrune.java index 9bba0e50..aa977587 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSPrune.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSPrune.java @@ -1,18 +1,47 @@ /* - HDFS Data Ingestion for PTH_06 use CFE-39 - Copyright (C) 2022 Fail-Safe IT Solutions Oy - - 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. + * HDFS Data Ingestion for PTH_06 use CFE-39 + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. */ package com.teragrep.cfe_39.consumers.kafka; diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSRead.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSRead.java index d00e17f2..1b8aef69 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSRead.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSRead.java @@ -1,3 +1,49 @@ +/* + * HDFS Data Ingestion for PTH_06 use CFE-39 + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. + */ + package com.teragrep.cfe_39.consumers.kafka; import com.teragrep.cfe_39.Config; diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java index 9a49f3ed..5a6bfcab 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java @@ -1,18 +1,47 @@ /* - HDFS Data Ingestion for PTH_06 use CFE-39 - Copyright (C) 2022 Fail-Safe IT Solutions Oy - - 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. + * HDFS Data Ingestion for PTH_06 use CFE-39 + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. */ package com.teragrep.cfe_39.consumers.kafka; diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java index c75c9c03..32500e77 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java @@ -1,18 +1,47 @@ /* - HDFS Data Ingestion for PTH_06 use CFE-39 - Copyright (C) 2022 Fail-Safe IT Solutions Oy - - 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. + * HDFS Data Ingestion for PTH_06 use CFE-39 + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. */ package com.teragrep.cfe_39.consumers.kafka; diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReader.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReader.java index 31b100e6..f34f2dde 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReader.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReader.java @@ -1,18 +1,47 @@ /* - HDFS Data Ingestion for PTH_06 use CFE-39 - Copyright (C) 2022 Fail-Safe IT Solutions Oy - - 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. + * HDFS Data Ingestion for PTH_06 use CFE-39 + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. */ package com.teragrep.cfe_39.consumers.kafka; diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java index 30153a1e..5e346aa4 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java @@ -1,18 +1,47 @@ /* - HDFS Data Ingestion for PTH_06 use CFE-39 - Copyright (C) 2022 Fail-Safe IT Solutions Oy - - 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. + * HDFS Data Ingestion for PTH_06 use CFE-39 + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. */ package com.teragrep.cfe_39.consumers.kafka; diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/NullOffset.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/NullOffset.java index 6727db68..73cee40e 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/NullOffset.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/NullOffset.java @@ -1,19 +1,49 @@ /* - HDFS Data Ingestion for PTH_06 use CFE-39 - Copyright (C) 2022 Fail-Safe IT Solutions Oy - - 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. + * HDFS Data Ingestion for PTH_06 use CFE-39 + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. */ + package com.teragrep.cfe_39.consumers.kafka; public class NullOffset extends Offset{ diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/Offset.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/Offset.java index 4a931166..80474527 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/Offset.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/Offset.java @@ -1,19 +1,49 @@ /* - HDFS Data Ingestion for PTH_06 use CFE-39 - Copyright (C) 2022 Fail-Safe IT Solutions Oy - - 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. + * HDFS Data Ingestion for PTH_06 use CFE-39 + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. */ + package com.teragrep.cfe_39.consumers.kafka; public abstract class Offset { diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinator.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinator.java index d053d6b0..0f29cb24 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinator.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinator.java @@ -1,18 +1,47 @@ /* - HDFS Data Ingestion for PTH_06 use CFE-39 - Copyright (C) 2022 Fail-Safe IT Solutions Oy - - 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. + * HDFS Data Ingestion for PTH_06 use CFE-39 + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. */ package com.teragrep.cfe_39.consumers.kafka; diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/RecordOffset.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/RecordOffset.java index b566c39e..216f7c97 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/RecordOffset.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/RecordOffset.java @@ -1,18 +1,47 @@ /* - HDFS Data Ingestion for PTH_06 use CFE-39 - Copyright (C) 2022 Fail-Safe IT Solutions Oy - - 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. + * HDFS Data Ingestion for PTH_06 use CFE-39 + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. */ package com.teragrep.cfe_39.consumers.kafka; diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java index 604374f8..248c608a 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java @@ -1,18 +1,47 @@ /* - HDFS Data Ingestion for PTH_06 use CFE-39 - Copyright (C) 2022 Fail-Safe IT Solutions Oy - - 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. + * HDFS Data Ingestion for PTH_06 use CFE-39 + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. */ package com.teragrep.cfe_39.consumers.kafka; diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/queue/QueueUtilities.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/queue/QueueUtilities.java index b37056db..887672e4 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/queue/QueueUtilities.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/queue/QueueUtilities.java @@ -1,18 +1,47 @@ /* - HDFS Data Ingestion for PTH_06 use CFE-39 - Copyright (C) 2022 Fail-Safe IT Solutions Oy - - 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. + * HDFS Data Ingestion for PTH_06 use CFE-39 + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. */ package com.teragrep.cfe_39.consumers.kafka.queue; diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/queue/WritableQueue.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/queue/WritableQueue.java index 80618adf..0ae38766 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/queue/WritableQueue.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/queue/WritableQueue.java @@ -1,18 +1,47 @@ /* - HDFS Data Ingestion for PTH_06 use CFE-39 - Copyright (C) 2022 Fail-Safe IT Solutions Oy - - 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. + * HDFS Data Ingestion for PTH_06 use CFE-39 + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. */ package com.teragrep.cfe_39.consumers.kafka.queue; diff --git a/src/main/java/com/teragrep/cfe_39/metrics/DurationStatistics.java b/src/main/java/com/teragrep/cfe_39/metrics/DurationStatistics.java index 244bcc7b..ac32fcd6 100644 --- a/src/main/java/com/teragrep/cfe_39/metrics/DurationStatistics.java +++ b/src/main/java/com/teragrep/cfe_39/metrics/DurationStatistics.java @@ -1,18 +1,47 @@ /* - HDFS Data Ingestion for PTH_06 use CFE-39 - Copyright (C) 2022 Fail-Safe IT Solutions Oy - - 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. + * HDFS Data Ingestion for PTH_06 use CFE-39 + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. */ package com.teragrep.cfe_39.metrics; diff --git a/src/main/java/com/teragrep/cfe_39/metrics/topic/TopicCounter.java b/src/main/java/com/teragrep/cfe_39/metrics/topic/TopicCounter.java index 7212045c..132925c0 100644 --- a/src/main/java/com/teragrep/cfe_39/metrics/topic/TopicCounter.java +++ b/src/main/java/com/teragrep/cfe_39/metrics/topic/TopicCounter.java @@ -1,18 +1,47 @@ /* - HDFS Data Ingestion for PTH_06 use CFE-39 - Copyright (C) 2022 Fail-Safe IT Solutions Oy - - 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. + * HDFS Data Ingestion for PTH_06 use CFE-39 + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. */ package com.teragrep.cfe_39.metrics.topic; diff --git a/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java b/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java index 30e850c3..172061b0 100644 --- a/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java +++ b/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java @@ -1,3 +1,49 @@ +/* + * HDFS Data Ingestion for PTH_06 use CFE-39 + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. + */ + package com.teragrep.cfe_39; import com.teragrep.cfe_39.avro.SyslogRecord; diff --git a/src/test/java/com/teragrep/cfe_39/HdfsTest.java b/src/test/java/com/teragrep/cfe_39/HdfsTest.java index 9de328eb..0f0582fc 100644 --- a/src/test/java/com/teragrep/cfe_39/HdfsTest.java +++ b/src/test/java/com/teragrep/cfe_39/HdfsTest.java @@ -1,3 +1,49 @@ +/* + * HDFS Data Ingestion for PTH_06 use CFE-39 + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. + */ + package com.teragrep.cfe_39; import com.teragrep.cfe_39.avro.SyslogRecord; diff --git a/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java b/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java index 3e5077aa..c2c3af35 100644 --- a/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java +++ b/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java @@ -1,3 +1,49 @@ +/* + * HDFS Data Ingestion for PTH_06 use CFE-39 + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. + */ + package com.teragrep.cfe_39; import com.teragrep.cfe_39.consumers.kafka.KafkaController; From 45a69d978cb0180d758cc8a6bc58ce2f27069df9 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Wed, 29 May 2024 14:05:32 +0300 Subject: [PATCH 072/146] Added spotless, enforcer and jacoco plugins and their requirements. --- eclipse-java-formatter.xml | 450 +++++++++++++++++++++++++++++++++++++ license-header | 45 ++++ pom.xml | 96 ++++++++ 3 files changed, 591 insertions(+) create mode 100644 eclipse-java-formatter.xml create mode 100644 license-header diff --git a/eclipse-java-formatter.xml b/eclipse-java-formatter.xml new file mode 100644 index 00000000..1e4e9905 --- /dev/null +++ b/eclipse-java-formatter.xml @@ -0,0 +1,450 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/license-header b/license-header new file mode 100644 index 00000000..d14a1f51 --- /dev/null +++ b/license-header @@ -0,0 +1,45 @@ +/* + * HDFS Data Ingestion for PTH_06 use CFE-39 + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. + */ \ No newline at end of file diff --git a/pom.xml b/pom.xml index d8d08cc1..eac3362b 100644 --- a/pom.xml +++ b/pom.xml @@ -48,6 +48,102 @@ 1.8 + + org.apache.maven.plugins + maven-enforcer-plugin + 3.4.1 + + + enforce + none + + + enforce-maven + + enforce + + + + + 3.2.5 + + + + + + + + com.diffplug.spotless + spotless-maven-plugin + 2.43.0 + + + + ${project.basedir}/eclipse-java-formatter.xml + 4.10.0 + + + + ${project.basedir}/license-header + + + + + + UTF-8 + \n + true + false + 2 + recommended_2008_06 + true + true + true + + + + + + .gitattributes + .gitignore + + + + + true + 4 + + + + + + + + check + + compile + + + + + org.jacoco + jacoco-maven-plugin + 0.8.12 + + + + prepare-agent + + + + report + prepare-package + + report + + + + From aa396686eb90fd6516b739351ad61de75b329e02 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Wed, 29 May 2024 14:09:10 +0300 Subject: [PATCH 073/146] Spotless --- pom.xml | 550 +++++++++--------- src/main/java/com/teragrep/cfe_39/Config.java | 37 +- .../consumers/kafka/DatabaseOutput.java | 154 +++-- .../cfe_39/consumers/kafka/HDFSPrune.java | 24 +- .../cfe_39/consumers/kafka/HDFSRead.java | 36 +- .../cfe_39/consumers/kafka/HDFSWrite.java | 28 +- .../consumers/kafka/KafkaController.java | 38 +- .../cfe_39/consumers/kafka/KafkaReader.java | 9 +- .../kafka/MockKafkaConsumerFactoryTemp.java | 274 +++++---- .../cfe_39/consumers/kafka/NullOffset.java | 3 +- .../cfe_39/consumers/kafka/Offset.java | 7 +- .../consumers/kafka/ReadCoordinator.java | 41 +- .../cfe_39/consumers/kafka/RecordOffset.java | 10 +- .../consumers/kafka/SyslogAvroWriter.java | 14 +- .../consumers/kafka/queue/QueueUtilities.java | 24 +- .../consumers/kafka/queue/WritableQueue.java | 32 +- .../cfe_39/metrics/DurationStatistics.java | 20 +- .../cfe_39/metrics/topic/TopicCounter.java | 6 +- .../com/teragrep/cfe_39/CombinedFullTest.java | 205 +++++-- .../java/com/teragrep/cfe_39/HdfsTest.java | 222 +++++-- .../teragrep/cfe_39/KafkaConsumerTest.java | 197 +++++-- 21 files changed, 1155 insertions(+), 776 deletions(-) diff --git a/pom.xml b/pom.xml index eac3362b..a66b3d0d 100644 --- a/pom.xml +++ b/pom.xml @@ -1,278 +1,274 @@ - - 4.0.0 - com.teragrep - cfe_39 - jar - ${revision}${sha1}${changelist} - cfe_39 - - UTF-8 - 1.8 - 1.8 - 1.8 - 0.0.1 - -SNAPSHOT - - 4.2.8 - 0.16.0 - 3.3.6 - - - - - org.apache.avro - avro-maven-plugin - 1.11.3 - - - generate-sources - - schema - - - ${project.basedir}/src/main/avro/ - ${project.basedir}/src/main/java/ - - - - - - org.apache.maven.plugins - maven-compiler-plugin - 3.8.1 - - 1.8 - 1.8 - - - - org.apache.maven.plugins - maven-enforcer-plugin - 3.4.1 - - - enforce - none - - - enforce-maven - - enforce - - - - - 3.2.5 - - - - - - - - com.diffplug.spotless - spotless-maven-plugin - 2.43.0 - - - - ${project.basedir}/eclipse-java-formatter.xml - 4.10.0 - - - - ${project.basedir}/license-header - - - - - - UTF-8 - \n - true - false - 2 - recommended_2008_06 - true - true - true - - - - - - .gitattributes - .gitignore - - - - - true - 4 - - - - - - - - check - - compile - - - - - org.jacoco - jacoco-maven-plugin - 0.8.12 - - - - prepare-agent - - - - report - prepare-package - - report - - - - - - - - - io.dropwizard.metrics - metrics-core - ${metrics.version} - - - io.dropwizard.metrics - metrics-jmx - ${metrics.version} - - - io.prometheus - simpleclient - ${prometheus-simpleclient.version} - - - io.prometheus - simpleclient_dropwizard - ${prometheus-simpleclient.version} - - - io.prometheus - simpleclient_servlet - ${prometheus-simpleclient.version} - - - io.prometheus - simpleclient_hotspot - ${prometheus-simpleclient.version} - - - org.junit.jupiter - junit-jupiter-engine - 5.7.1 - test - - - org.junit.platform - junit-platform-launcher - 1.7.1 - test - - - org.junit.jupiter - junit-jupiter-api - 5.7.1 - test - - - org.junit.jupiter - junit-jupiter - 5.7.1 - test - - - com.teragrep - rlo_06 - 9.0.1 - - - - org.apache.kafka - kafka-clients - 3.4.0 - - - - org.xerial.snappy - snappy-java - 1.1.10.5 - - - org.apache.avro - avro - 1.11.3 - - - - org.apache.hadoop - hadoop-client - ${hadoop.version} - - - - org.apache.hadoop - hadoop-minicluster - ${hadoop.version} - test - - - - org.apache.hadoop - hadoop-common - ${hadoop.version} - - - - org.apache.hadoop - hadoop-hdfs - ${hadoop.version} - test - - - - org.mockito - mockito-core - 4.11.0 - test - - - - - - org.apache.logging.log4j - log4j-slf4j2-impl - 2.20.0 - - - org.apache.logging.log4j - log4j-core - 2.20.0 - - - org.slf4j - slf4j-api - 2.0.7 - - - \ No newline at end of file + + 4.0.0 + com.teragrep + cfe_39 + ${revision}${sha1}${changelist} + jar + cfe_39 + + -SNAPSHOT + 3.3.6 + 1.8 + 1.8 + 1.8 + 4.2.8 + UTF-8 + 0.16.0 + 0.0.1 + + + + + io.dropwizard.metrics + metrics-core + ${metrics.version} + + + io.dropwizard.metrics + metrics-jmx + ${metrics.version} + + + io.prometheus + simpleclient + ${prometheus-simpleclient.version} + + + io.prometheus + simpleclient_dropwizard + ${prometheus-simpleclient.version} + + + io.prometheus + simpleclient_servlet + ${prometheus-simpleclient.version} + + + io.prometheus + simpleclient_hotspot + ${prometheus-simpleclient.version} + + + org.junit.jupiter + junit-jupiter-engine + 5.7.1 + test + + + org.junit.platform + junit-platform-launcher + 1.7.1 + test + + + org.junit.jupiter + junit-jupiter-api + 5.7.1 + test + + + org.junit.jupiter + junit-jupiter + 5.7.1 + test + + + com.teragrep + rlo_06 + 9.0.1 + + + + org.apache.kafka + kafka-clients + 3.4.0 + + + + org.xerial.snappy + snappy-java + 1.1.10.5 + + + org.apache.avro + avro + 1.11.3 + + + + org.apache.hadoop + hadoop-client + ${hadoop.version} + + + + org.apache.hadoop + hadoop-minicluster + ${hadoop.version} + test + + + + org.apache.hadoop + hadoop-common + ${hadoop.version} + + + + org.apache.hadoop + hadoop-hdfs + ${hadoop.version} + test + + + + org.mockito + mockito-core + 4.11.0 + test + + + + org.apache.logging.log4j + log4j-slf4j2-impl + 2.20.0 + + + org.apache.logging.log4j + log4j-core + 2.20.0 + + + org.slf4j + slf4j-api + 2.0.7 + + + + + + org.apache.avro + avro-maven-plugin + 1.11.3 + + + + schema + + generate-sources + + ${project.basedir}/src/main/avro/ + ${project.basedir}/src/main/java/ + + + + + + org.apache.maven.plugins + maven-compiler-plugin + 3.8.1 + + 1.8 + 1.8 + + + + org.apache.maven.plugins + maven-enforcer-plugin + 3.4.1 + + + enforce-maven + + enforce + + + + + 3.2.5 + + + + + + enforce + none + + + + + com.diffplug.spotless + spotless-maven-plugin + 2.43.0 + + + + ${project.basedir}/eclipse-java-formatter.xml + 4.10.0 + + + + ${project.basedir}/license-header + + + + + + UTF-8 + \n + true + false + 2 + recommended_2008_06 + true + true + true + + + + + + .gitattributes + .gitignore + + + + + true + 4 + + + + + + + + check + + compile + + + + + org.jacoco + jacoco-maven-plugin + 0.8.12 + + + + prepare-agent + + + + report + + report + + prepare-package + + + + + + diff --git a/src/main/java/com/teragrep/cfe_39/Config.java b/src/main/java/com/teragrep/cfe_39/Config.java index c187b200..2c079feb 100644 --- a/src/main/java/com/teragrep/cfe_39/Config.java +++ b/src/main/java/com/teragrep/cfe_39/Config.java @@ -43,10 +43,8 @@ * Teragrep, the applicable Commercial License may apply to this file if you as * a licensee so wish it. */ - package com.teragrep.cfe_39; -import org.apache.logging.log4j.core.config.ConfigurationSource; import org.apache.logging.log4j.core.config.Configurator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -61,6 +59,7 @@ import java.util.Properties; public class Config { + private final String queueTopicPattern; private final Properties kafkaConsumerProperties; private static final Logger LOGGER = LoggerFactory.getLogger(Config.class); @@ -84,10 +83,11 @@ public class Config { Config() throws IOException { Properties properties = new Properties(); - Path configPath = Paths.get(System.getProperty("cfe_30.config.location", System.getProperty("user.dir") + "/etc/application.properties")); + Path configPath = Paths + .get(System.getProperty("cfe_30.config.location", System.getProperty("user.dir") + "/etc/application.properties")); LOGGER.info("Loading application config <[{}]>", configPath.toAbsolutePath()); - try(InputStream inputStream = Files.newInputStream(configPath)) { + try (InputStream inputStream = Files.newInputStream(configPath)) { properties.load(inputStream); LOGGER.debug("Got configuration: <{}>", properties); } @@ -114,23 +114,23 @@ public class Config { this.kerberosKeytabPath = properties.getProperty("KerberosKeytabPath", ""); this.kerberosTestMode = properties.getProperty("dfs.client.use.datanode.hostname", "false"); - // kafka this.queueTopicPattern = properties.getProperty("queueTopicPattern", "^.*$"); this.numOfConsumers = Integer.parseInt(properties.getProperty("numOfConsumers", "1")); this.kafkaConsumerProperties = loadSubProperties(properties, "consumer."); String loginConfig = properties.getProperty("java.security.auth.login.config"); - if(loginConfig == null) { + if (loginConfig == null) { throw new IOException("Property java.security.auth.login.config does not exist"); } - if(!(new File(loginConfig)).isFile()) { + if (!(new File(loginConfig)).isFile()) { throw new IOException("File '" + loginConfig + "' set by java.security.auth.login.config does not exist"); } System.setProperty("java.security.auth.login.config", loginConfig); // Just for loggers to work - Path log4j2Config = Paths.get(properties.getProperty("log4j2.configurationFile", System.getProperty("user.dir") + "/etc/log4j2.properties")); + Path log4j2Config = Paths + .get(properties.getProperty("log4j2.configurationFile", System.getProperty("user.dir") + "/etc/log4j2.properties")); LOGGER.info("Loading log4j2 config from <[{}]>", log4j2Config.toRealPath()); Configurator.reconfigure(log4j2Config.toUri()); } @@ -143,7 +143,7 @@ private Properties loadSubProperties(Properties properties, String prefix) { String key = String.valueOf(keys.nextElement()); if (key.startsWith(prefix)) { String value = properties.getProperty(key); - String subKey = key.replaceFirst(prefix,""); + String subKey = key.replaceFirst(prefix, ""); subProperties.put(subKey, value); } } @@ -153,9 +153,11 @@ private Properties loadSubProperties(Properties properties, String prefix) { public String getHdfsPath() { return hdfsPath; } + public void setHdfsuri(String input) { this.hdfsuri = input; } + public String getHdfsuri() { return hdfsuri; } @@ -163,49 +165,64 @@ public String getHdfsuri() { public String getQueueDirectory() { return queueDirectory; } + public String getQueueNamePrefix() { return queueNamePrefix; } + public String getQueueTopicPattern() { return queueTopicPattern; } + public Properties getKafkaConsumerProperties() { return kafkaConsumerProperties; } + public String getKerberosHost() { return kerberosHost; } + public String getKerberosRealm() { return kerberosRealm; } + public String getKerberosPrincipal() { return kerberosPrincipal; } + public String getHadoopAuthentication() { return hadoopAuthentication; } + public String getHadoopAuthorization() { return hadoopAuthorization; } + public String getKerberosKeytabUser() { return kerberosKeytabUser; } + public String getKerberosKeytabPath() { return kerberosKeytabPath; } + public String getKerberosTestMode() { return kerberosTestMode; } + public long getMaximumFileSize() { return maximumFileSize; } + public void setMaximumFileSize(long maximumFileSize) { this.maximumFileSize = maximumFileSize; } + public int getNumOfConsumers() { return numOfConsumers; } + public long getPruneOffset() { return pruneOffset; } -} \ No newline at end of file +} diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java index 28475126..bb714c1d 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java @@ -43,7 +43,6 @@ * Teragrep, the applicable Commercial License may apply to this file if you as * a licensee so wish it. */ - package com.teragrep.cfe_39.consumers.kafka; import com.teragrep.cfe_39.Config; @@ -69,12 +68,12 @@ The mock consumer is activated for testing using the configuration file: readerKafkaProperties.getProperty("useMockKafkaConsumer", "false")*/ public class DatabaseOutput implements Consumer> { + private static final Logger LOGGER = LoggerFactory.getLogger(DatabaseOutput.class); private final RFC5424Frame rfc5424Frame = new RFC5424Frame(false); private final String table; - private final DurationStatistics durationStatistics; private final TopicCounter topicCounter; @@ -97,12 +96,7 @@ public class DatabaseOutput implements Consumer> { private final Config config; private long epochMicros_last; - DatabaseOutput( - Config config, - String table, - DurationStatistics durationStatistics, - TopicCounter topicCounter - ) { + DatabaseOutput(Config config, String table, DurationStatistics durationStatistics, TopicCounter topicCounter) { this.config = config; this.table = table; this.durationStatistics = durationStatistics; @@ -110,20 +104,18 @@ public class DatabaseOutput implements Consumer> { this.maximumFileSize = config.getMaximumFileSize(); // queueDirectory and queueNamePrefix are only used for temporarily storing the AVRO-serialized files before committing them to HDFS when the file size reaches the threshold (or all records are processed). - this.writableQueue = new WritableQueue( - config.getQueueDirectory() - ); - - this.sourceConcatenationBuffer = ByteBuffer.allocateDirect(256*1024); - teragrepStreamName = new SDVector("teragrep@48577","streamname"); - teragrepDirectory = new SDVector("teragrep@48577","directory"); - this.eventNodeSourceSource = new SDVector("event_node_source@48577","source"); - this.eventNodeRelaySource = new SDVector("event_node_relay@48577","source"); - this.eventNodeSourceSourceModule = new SDVector("event_node_source@48577","source_module"); - this.eventNodeRelaySourceModule = new SDVector("event_node_relay@48577","source_module"); - this.eventNodeSourceHostname = new SDVector("event_node_source@48577","hostname"); - this.eventNodeRelayHostname = new SDVector("event_node_relay@48577","hostname"); - this.originHostname = new SDVector("origin@48577","hostname"); + this.writableQueue = new WritableQueue(config.getQueueDirectory()); + + this.sourceConcatenationBuffer = ByteBuffer.allocateDirect(256 * 1024); + teragrepStreamName = new SDVector("teragrep@48577", "streamname"); + teragrepDirectory = new SDVector("teragrep@48577", "directory"); + this.eventNodeSourceSource = new SDVector("event_node_source@48577", "source"); + this.eventNodeRelaySource = new SDVector("event_node_relay@48577", "source"); + this.eventNodeSourceSourceModule = new SDVector("event_node_source@48577", "source_module"); + this.eventNodeRelaySourceModule = new SDVector("event_node_relay@48577", "source_module"); + this.eventNodeSourceHostname = new SDVector("event_node_source@48577", "hostname"); + this.eventNodeRelayHostname = new SDVector("event_node_relay@48577", "hostname"); + this.originHostname = new SDVector("origin@48577", "hostname"); this.epochMicros_last = 0L; } @@ -141,12 +133,13 @@ boolean committedToHdfs(long fileSize, RecordOffset recordOffsetObject) { } // This part defines a new empty file to which the new AVRO-serialized records are stored until it again hits the 64M size limit. - writableQueue.setQueueNamePrefix(recordOffsetObject.getTopic()+recordOffsetObject.getPartition()); + writableQueue.setQueueNamePrefix(recordOffsetObject.getTopic() + recordOffsetObject.getPartition()); syslogFile = writableQueue.getNextWritableFile(); syslogAvroWriter = new SyslogAvroWriter(syslogFile); return true; } - } catch (IOException ioException) { + } + catch (IOException ioException) { throw new UncheckedIOException(ioException); } return false; @@ -172,7 +165,11 @@ public void accept(List recordOffsetObjectList) { long thisTime = Instant.now().toEpochMilli(); long ftook = thisTime - lastTimeCalled; topicCounter.setKafkaLatency(ftook); - LOGGER.debug("Fuura searching your batch for <[{}]> with records <{}> and took <{}> milliseconds. <{}> EPS. ", table, recordOffsetObjectList.size(), (ftook), (recordOffsetObjectList.size() * 1000L / ftook)); + LOGGER + .debug( + "Fuura searching your batch for <[{}]> with records <{}> and took <{}> milliseconds. <{}> EPS. ", + table, recordOffsetObjectList.size(), (ftook), (recordOffsetObjectList.size() * 1000L / ftook) + ); long batchBytes = 0L; /* The recordOffsetObjectList loop will go through all the objects in the list. @@ -187,37 +184,57 @@ public void accept(List recordOffsetObjectList) { // Initializing syslogAvroWriter and lastObject. if (syslogAvroWriter == null && lastObject.isNull()) { try { - writableQueue.setQueueNamePrefix(recordOffsetObject.getTopic()+recordOffsetObject.getPartition()); - syslogFile = - writableQueue.getNextWritableFile(); + writableQueue.setQueueNamePrefix(recordOffsetObject.getTopic() + recordOffsetObject.getPartition()); + syslogFile = writableQueue.getNextWritableFile(); // The HDFS filename is only finalized when the AVRO-serialized file is finalized, because every Kafka-record added to the file is going to change the offset that is going to be used for the filename. syslogAvroWriter = new SyslogAvroWriter(syslogFile); lastObject = recordOffsetObject; - } catch (IOException ioException) { + } + catch (IOException ioException) { throw new IllegalArgumentException(ioException); } - } else { + } + else { try { - if (lastObject.getTopic().equals(recordOffsetObject.getTopic()) & lastObject.getPartition().equals(recordOffsetObject.getPartition())) { + if ( + lastObject.getTopic().equals(recordOffsetObject.getTopic()) + & lastObject.getPartition().equals(recordOffsetObject.getPartition()) + ) { // Records left to consume in the current partition. - boolean fileCommitted = committedToHdfs(syslogAvroWriter.getFileSize(), (RecordOffset) lastObject); + boolean fileCommitted = committedToHdfs( + syslogAvroWriter.getFileSize(), (RecordOffset) lastObject + ); if (fileCommitted) { - LOGGER.debug("Target file size reached, file <{}> stored to <{}> in HDFS", syslogFile.getName(), lastObject.getTopic() + "/" + lastObject.getPartition() + "." + lastObject.getOffset()); - } else { - LOGGER.debug("Target file size not yet reached, continuing writing records to <{}>.", syslogFile.getName()); + LOGGER + .debug( + "Target file size reached, file <{}> stored to <{}> in HDFS", + syslogFile.getName(), + lastObject.getTopic() + "/" + lastObject.getPartition() + "." + + lastObject.getOffset() + ); + } + else { + LOGGER + .debug( + "Target file size not yet reached, continuing writing records to <{}>.", + syslogFile.getName() + ); } - } else { + } + else { // Previous partition was fully consumed. Commit file to HDFS and create a new AVRO-file. syslogAvroWriter.close(); HDFSWrite writer = new HDFSWrite(config, (RecordOffset) lastObject); writer.commit(syslogFile, epochMicros_last); // This part defines a new empty file to which the new AVRO-serialized records are stored until it again hits the 64M size limit. - writableQueue.setQueueNamePrefix(recordOffsetObject.getTopic() + recordOffsetObject.getPartition()); + writableQueue + .setQueueNamePrefix(recordOffsetObject.getTopic() + recordOffsetObject.getPartition()); syslogFile = writableQueue.getNextWritableFile(); syslogAvroWriter = new SyslogAvroWriter(syslogFile); } - } catch (IOException ioException) { + } + catch (IOException ioException) { throw new UncheckedIOException(ioException); } } @@ -227,8 +244,8 @@ public void accept(List recordOffsetObjectList) { InputStream inputStream = new ByteArrayInputStream(byteArray); rfc5424Frame.load(inputStream); try { - if(rfc5424Frame.next()) { -/* rfc5424Frame has loaded the record data, it's ready for deserialization. + if (rfc5424Frame.next()) { + /* rfc5424Frame has loaded the record data, it's ready for deserialization. Implement AVRO serialization for the Kafka records here, preparing the data for writing to HDFS. Write all the data into a file using AVRO. The size of each AVRO-serialized file should be as close to 64M as possible.*/ @@ -240,8 +257,11 @@ public void accept(List recordOffsetObjectList) { final byte[] origin = eventToOrigin(); // Format: Use AVRO format with syslog columns as indexed ones - final long epochMicros = rfc3339ToEpoch(new RFC5424Timestamp(rfc5424Frame.timestamp).toZonedDateTime()); - SyslogRecord syslogRecord = SyslogRecord.newBuilder() + final long epochMicros = rfc3339ToEpoch( + new RFC5424Timestamp(rfc5424Frame.timestamp).toZonedDateTime() + ); + SyslogRecord syslogRecord = SyslogRecord + .newBuilder() .setTimestamp(epochMicros) .setPayload(rfc5424Frame.msg.toString()) .setDirectory(rfc5424Frame.structuredData.getValue(teragrepDirectory).toString()) @@ -253,7 +273,6 @@ public void accept(List recordOffsetObjectList) { .setOrigin(new String(origin, StandardCharsets.UTF_8)) .build(); - // Calculate the size of syslogRecord that is going to be written to syslogAvroWriter-file. long capacity = syslogRecord.toByteBuffer().capacity(); // handle initial timestamp @@ -261,11 +280,24 @@ public void accept(List recordOffsetObjectList) { epochMicros_last = epochMicros; } // Check if there is still room in syslogAvroWriter for another syslogRecord. Commit syslogAvroWriter to HDFS if no room left, emptying it out in the process. - boolean fileCommitted = committedToHdfs(syslogAvroWriter.getFileSize() + capacity, (RecordOffset) lastObject); + boolean fileCommitted = committedToHdfs( + syslogAvroWriter.getFileSize() + capacity, (RecordOffset) lastObject + ); if (fileCommitted) { - LOGGER.debug("Target file size reached, file <{}> stored to <{}> in HDFS", syslogFile.getName(), lastObject.getTopic()+"/"+lastObject.getPartition()+"."+lastObject.getOffset()); - }else { - LOGGER.debug("Target file size not yet reached, continuing writing records to <{}>.", syslogFile.getName()); + LOGGER + .debug( + "Target file size reached, file <{}> stored to <{}> in HDFS", + syslogFile.getName(), + lastObject.getTopic() + "/" + lastObject.getPartition() + "." + + lastObject.getOffset() + ); + } + else { + LOGGER + .debug( + "Target file size not yet reached, continuing writing records to <{}>.", + syslogFile.getName() + ); } // if more records can be inserted, update epochMicros_last with the timestamp of the last inserted record. epochMicros_last = epochMicros; @@ -273,7 +305,8 @@ public void accept(List recordOffsetObjectList) { syslogAvroWriter.write(syslogRecord); lastObject = recordOffsetObject; } - } catch (IOException e) { + } + catch (IOException e) { throw new RuntimeException(e); } } @@ -286,7 +319,8 @@ public void accept(List recordOffsetObjectList) { writer.commit(syslogFile, epochMicros_last); // commits the final AVRO-file to HDFS. } } - } catch (IOException e) { + } + catch (IOException e) { throw new RuntimeException(e); } @@ -311,12 +345,14 @@ public void accept(List recordOffsetObjectList) { topicCounter.addToTotalBytes(batchBytes); topicCounter.addToTotalRecords(recordOffsetObjectList.size()); - LOGGER.debug("Sent batch for <[{}]> with records <{}> and size <{}> KB took <{}> milliseconds. <{}> RPS. <{}> KB/s ", table, recordOffsetObjectList.size(), batchBytes / 1024, (took), rps, bps / 1024); + LOGGER + .debug( + "Sent batch for <[{}]> with records <{}> and size <{}> KB took <{}> milliseconds. <{}> RPS. <{}> KB/s ", + table, recordOffsetObjectList.size(), batchBytes / 1024, (took), rps, bps / 1024 + ); lastTimeCalled = Instant.now().toEpochMilli(); } - - private byte[] eventToOrigin() { byte[] origin; Fragment originFragment = rfc5424Frame.structuredData.getValue(originHostname); @@ -324,7 +360,7 @@ private byte[] eventToOrigin() { origin = originFragment.toBytes(); } else { - origin = new byte[]{}; + origin = new byte[] {}; } return origin; } @@ -347,7 +383,7 @@ private byte[] eventToSource() { source_module = sourceModuleFragment.toBytes(); } else { - source_module = new byte[]{}; + source_module = new byte[] {}; } Fragment sourceHostnameFragment = rfc5424Frame.structuredData.getValue(eventNodeSourceHostname); @@ -360,10 +396,9 @@ private byte[] eventToSource() { source_hostname = sourceHostnameFragment.toBytes(); } else { - source_hostname = new byte[]{}; + source_hostname = new byte[] {}; } - Fragment sourceSourceFragment = rfc5424Frame.structuredData.getValue(eventNodeSourceSource); if (sourceHostnameFragment.isStub) { sourceSourceFragment = rfc5424Frame.structuredData.getValue(eventNodeRelaySource); @@ -374,14 +409,13 @@ private byte[] eventToSource() { source_source = sourceSourceFragment.toBytes(); } else { - source_source = new byte[]{}; + source_source = new byte[] {}; } - sourceConcatenationBuffer.put(source_module); sourceConcatenationBuffer.put((byte) ':'); sourceConcatenationBuffer.put(source_hostname); - sourceConcatenationBuffer.put((byte)':'); + sourceConcatenationBuffer.put((byte) ':'); sourceConcatenationBuffer.put(source_source); sourceConcatenationBuffer.flip(); @@ -390,4 +424,4 @@ private byte[] eventToSource() { return input; } -} \ No newline at end of file +} diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSPrune.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSPrune.java index aa977587..3853093e 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSPrune.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSPrune.java @@ -43,7 +43,6 @@ * Teragrep, the applicable Commercial License may apply to this file if you as * a licensee so wish it. */ - package com.teragrep.cfe_39.consumers.kafka; import com.teragrep.cfe_39.Config; @@ -60,9 +59,9 @@ import java.io.IOException; import java.net.URI; import java.util.Properties; -import java.util.concurrent.TimeUnit; public class HDFSPrune { + private static final Logger LOGGER = LoggerFactory.getLogger(HDFSPrune.class); private Config config; private final FileSystem fs; @@ -74,9 +73,8 @@ public HDFSPrune(Config config, String topicName) throws IOException { // Check for testmode from config. Properties readerKafkaProperties = config.getKafkaConsumerProperties(); - this.useMockKafkaConsumer = Boolean.parseBoolean( - readerKafkaProperties.getProperty("useMockKafkaConsumer", "false") - ); + this.useMockKafkaConsumer = Boolean + .parseBoolean(readerKafkaProperties.getProperty("useMockKafkaConsumer", "false")); if (useMockKafkaConsumer) { this.config = config; @@ -103,7 +101,8 @@ public HDFSPrune(Config config, String topicName) throws IOException { fs.mkdirs(newDirectoryPath); LOGGER.info("Path <{}> created.", path); } - }else { + } + else { // Code for initializing the class with kerberos. String hdfsuri = config.getHdfsuri(); // Get from config. @@ -155,14 +154,15 @@ public void prune() throws IOException { // Fetch the filestatuses of HDFS files. FileStatus[] fileStatuses = fs.listStatus(new Path(newDirectoryPath + "/")); if (fileStatuses.length > 0) { - for (FileStatus a : fileStatuses) { - // Delete old files - if (a.getModificationTime() < cutOffEpoch) { - boolean delete = fs.delete(a.getPath(), true); - LOGGER.info("Deleted file <{}>", a.getPath()); + for (FileStatus a : fileStatuses) { + // Delete old files + if (a.getModificationTime() < cutOffEpoch) { + boolean delete = fs.delete(a.getPath(), true); + LOGGER.info("Deleted file <{}>", a.getPath()); + } } } - } else { + else { LOGGER.info("No files found in directory <{}>", new Path(newDirectoryPath + "/")); } } diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSRead.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSRead.java index 1b8aef69..8517c106 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSRead.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSRead.java @@ -43,7 +43,6 @@ * Teragrep, the applicable Commercial License may apply to this file if you as * a licensee so wish it. */ - package com.teragrep.cfe_39.consumers.kafka; import com.teragrep.cfe_39.Config; @@ -77,9 +76,8 @@ The offset map can then be used for kafka consumer seek() method, which will add public HDFSRead(Config config) throws IOException { // Check if mock kafka consumer is enabled in the config. Properties readerKafkaProperties = config.getKafkaConsumerProperties(); - this.useMockKafkaConsumer = Boolean.parseBoolean( - readerKafkaProperties.getProperty("useMockKafkaConsumer", "false") - ); + this.useMockKafkaConsumer = Boolean + .parseBoolean(readerKafkaProperties.getProperty("useMockKafkaConsumer", "false")); if (useMockKafkaConsumer) { // Code for initializing the class in test mode without kerberos. @@ -99,12 +97,13 @@ public HDFSRead(Config config) throws IOException { // filesystem for HDFS access is set here try { fs = FileSystem.get(URI.create(hdfsuri), conf); - } catch (IOException e) { + } + catch (IOException e) { throw new RuntimeException(e); } - - }else { + } + else { // Code for initializing the class with kerberos. hdfsuri = config.getHdfsuri(); // Get from config.' path = config.getHdfsPath(); @@ -146,9 +145,9 @@ public Map hdfsStartOffsets() throws IOException { topicsRegexString = "^.*$"; // FIXME: all topics if none given } - Path workingDir=fs.getWorkingDirectory(); - Path newDirectoryPath= new Path(path); - if(!fs.exists(newDirectoryPath)) { + Path workingDir = fs.getWorkingDirectory(); + Path newDirectoryPath = new Path(path); + if (!fs.exists(newDirectoryPath)) { // Create new Directory fs.mkdirs(newDirectoryPath); LOGGER.info("Path <{}> created.", path); @@ -168,34 +167,37 @@ public Map hdfsStartOffsets() throws IOException { String offset = split[1]; TopicPartition topicPartition = new TopicPartition(topic, Integer.parseInt(partition)); if (!offsets.containsKey(topicPartition)) { - offsets.put(topicPartition, Long.parseLong(offset)+1); - } else { - if (offsets.get(topicPartition) < Long.parseLong(offset)+1) { - offsets.replace(topicPartition, Long.parseLong(offset)+1); + offsets.put(topicPartition, Long.parseLong(offset) + 1); + } + else { + if (offsets.get(topicPartition) < Long.parseLong(offset) + 1) { + offsets.replace(topicPartition, Long.parseLong(offset) + 1); } } } } - }else { + } + else { LOGGER.info("No matching directories found"); } return offsets; } private static final PathFilter topicFilter = new PathFilter() { + @Override public boolean accept(Path path) { return path.getName().matches(topicsRegexString); // Catches the directory names. } }; - // try-with-resources handles closing the filesystem automatically. public void close() { if (fs != null) { try { fs.close(); - } catch (IOException e) { + } + catch (IOException e) { throw new RuntimeException(e); } } diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java index 5a6bfcab..2df8b2b4 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java @@ -43,7 +43,6 @@ * Teragrep, the applicable Commercial License may apply to this file if you as * a licensee so wish it. */ - package com.teragrep.cfe_39.consumers.kafka; import com.teragrep.cfe_39.Config; @@ -57,11 +56,9 @@ import java.io.File; import java.io.IOException; import java.net.URI; -import java.nio.file.Files; -import java.nio.file.Paths; import java.util.Properties; -public class HDFSWrite implements AutoCloseable{ +public class HDFSWrite implements AutoCloseable { private static final Logger LOGGER = LoggerFactory.getLogger(HDFSWrite.class); private final String fileName; @@ -74,9 +71,8 @@ public class HDFSWrite implements AutoCloseable{ public HDFSWrite(Config config, RecordOffset lastObject) throws IOException { Properties readerKafkaProperties = config.getKafkaConsumerProperties(); - this.useMockKafkaConsumer = Boolean.parseBoolean( - readerKafkaProperties.getProperty("useMockKafkaConsumer", "false") - ); + this.useMockKafkaConsumer = Boolean + .parseBoolean(readerKafkaProperties.getProperty("useMockKafkaConsumer", "false")); if (useMockKafkaConsumer) { // Code for initializing the class for mock hdfs database usage without kerberos. @@ -85,8 +81,8 @@ public HDFSWrite(Config config, RecordOffset lastObject) throws IOException { /* The filepath should be something like hdfs:///opt/teragrep/cfe_39/srv/topic_name/0.12345 where 12345 is offset and 0 the partition. In other words the directory named topic_name holds files that are named and arranged based on partition and the partition's offset. Every partition has its own set of unique offset values. These values should be fetched from config and other input parameters (topic+partition+offset).*/ - path = config.getHdfsPath()+"/"+lastObject.topic; - fileName = lastObject.partition+"."+lastObject.offset; // filename should be constructed from partition and offset. + path = config.getHdfsPath() + "/" + lastObject.topic; + fileName = lastObject.partition + "." + lastObject.offset; // filename should be constructed from partition and offset. // ====== Init HDFS File System Object conf = new Configuration(); @@ -101,12 +97,13 @@ These values should be fetched from config and other input parameters (topic+par // filesystem for HDFS access is set here try { fs = FileSystem.get(URI.create(hdfsuri), conf); - } catch (IOException e) { + } + catch (IOException e) { throw new RuntimeException(e); } - - }else { + } + else { // Code for initializing the class for kerberized HDFS database usage. hdfsuri = config.getHdfsuri(); @@ -162,7 +159,8 @@ public void commit(File syslogFile, long lastEpochMicros) { Path hdfswritepath = new Path(newDirectoryPath.toString() + "/" + fileName); // filename should be set according to the requirements: 0.12345 where 0 is Kafka partition and 12345 is Kafka offset. if (fs.exists(hdfswritepath)) { throw new RuntimeException("File " + fileName + " already exists"); - } else { + } + else { LOGGER.info("Path <{}> doesn't exist.", path); } @@ -172,7 +170,8 @@ public void commit(File syslogFile, long lastEpochMicros) { boolean delete = syslogFile.delete(); // deletes the avro-file from the local disk now that it has been committed to HDFS. LOGGER.info("\nFile committed to HDFS, file writepath should be: <{}>\n", hdfswritepath); - } catch (IOException e) { + } + catch (IOException e) { throw new RuntimeException(e); } } @@ -183,5 +182,4 @@ public void close() { When used here fs.close() doesn't just affect the current class, it affects all the FileSystem objects that were created using FileSystem.get(URI.create(hdfsuri), conf); in different threads.*/ } - } diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java index 32500e77..7b208850 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java @@ -43,7 +43,6 @@ * Teragrep, the applicable Commercial License may apply to this file if you as * a licensee so wish it. */ - package com.teragrep.cfe_39.consumers.kafka; import com.teragrep.cfe_39.Config; @@ -56,7 +55,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; - import java.io.IOException; import java.sql.SQLException; import java.time.Duration; @@ -66,7 +64,6 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; - public class KafkaController { private static final Logger LOGGER = LoggerFactory.getLogger(KafkaController.class); @@ -85,13 +82,17 @@ public KafkaController(Config config) { this.config = config; Properties readerKafkaProperties = config.getKafkaConsumerProperties(); this.numOfConsumers = config.getNumOfConsumers(); - this.useMockKafkaConsumer = Boolean.parseBoolean( - readerKafkaProperties.getProperty("useMockKafkaConsumer", "false") - ); + this.useMockKafkaConsumer = Boolean + .parseBoolean(readerKafkaProperties.getProperty("useMockKafkaConsumer", "false")); if (useMockKafkaConsumer) { this.kafkaConsumer = MockKafkaConsumerFactoryTemp.getConsumer(0); // A consumer used only for scanning the available topics to be allocated to consumers running in different threads (thus 0 as input parameter). - } else { - this.kafkaConsumer = new KafkaConsumer<>(config.getKafkaConsumerProperties(), new ByteArrayDeserializer(), new ByteArrayDeserializer()); + } + else { + this.kafkaConsumer = new KafkaConsumer<>( + config.getKafkaConsumerProperties(), + new ByteArrayDeserializer(), + new ByteArrayDeserializer() + ); } hdfsStartOffsets = new HashMap<>(); } @@ -105,10 +106,11 @@ public void run() throws InterruptedException { List topicCounters = new CopyOnWriteArrayList<>(); // Generates offsets of the already committed records for Kafka and passes them to the kafka consumers. - try (HDFSRead hr = new HDFSRead(config) ) { + try (HDFSRead hr = new HDFSRead(config)) { hdfsStartOffsets = hr.hdfsStartOffsets(); LOGGER.debug("topicPartitionStartMap generated succesfully: <{}>", hdfsStartOffsets); - } catch (IOException e) { + } + catch (IOException e) { throw new RuntimeException(e); } @@ -126,7 +128,8 @@ public void run() throws InterruptedException { try { HDFSPrune hdfsPrune = new HDFSPrune(config, topic_name); hdfsPrune.prune(); - } catch (IOException e) { + } + catch (IOException e) { throw new RuntimeException(e); } } @@ -141,7 +144,8 @@ public void run() throws InterruptedException { } // Creates kafka topic consumer based on input parameters. - private void createReader(String topic, List listPartitionInfo, List topicCounters) throws SQLException { + private void createReader(String topic, List listPartitionInfo, List topicCounters) + throws SQLException { // Create a new topicCounter object for the topic that has not been added to topicCounters-list yet. TopicCounter topicCounter = new TopicCounter(topic); @@ -164,7 +168,7 @@ private void createReader(String topic, List listPartitionInfo, L output, hdfsStartOffsets ); - Thread readThread = new Thread(null, readCoordinator, topic+testi); // Starts the thread with readCoordinator that creates the consumer and subscribes to the topic. + Thread readThread = new Thread(null, readCoordinator, topic + testi); // Starts the thread with readCoordinator that creates the consumer and subscribes to the topic. threads.add(readThread); readThread.start(); // Starts the thread, in other words proceeds to call run() function of ReadCoordinator. } @@ -174,7 +178,7 @@ private void createReader(String topic, List listPartitionInfo, L private void topicScan(DurationStatistics durationStatistics, List topicCounters) { Map> listTopics = kafkaConsumer.listTopics(Duration.ofSeconds(60)); Pattern topicsRegex = Pattern.compile(config.getQueueTopicPattern()); -// Find the topics available in Kafka based on given QueueTopicPattern, both active and in-active. + // Find the topics available in Kafka based on given QueueTopicPattern, both active and in-active. Set foundTopics = new HashSet<>(); Map> foundPartitions = new HashMap<>(); for (Map.Entry> entry : listTopics.entrySet()) { @@ -185,7 +189,7 @@ private void topicScan(DurationStatistics durationStatistics, List } } if (foundTopics.isEmpty()) { - throw new IllegalStateException("Pattern <[" + config.getQueueTopicPattern() + "]> found no topics." ); + throw new IllegalStateException("Pattern <[" + config.getQueueTopicPattern() + "]> found no topics."); } // subtract currently active topics from found topics foundTopics.removeAll(activeTopics); @@ -194,7 +198,6 @@ private void topicScan(DurationStatistics durationStatistics, List foundPartitions.remove(topic_name); // removes the partitions from the list based on the topic name. } - // Activate all the found in-active topics, in other words create consumer groups for all of them using the createReader()-function. foundPartitions.forEach((k, v) -> { LOGGER.debug("Activating topic <{}>", k); @@ -202,7 +205,8 @@ private void topicScan(DurationStatistics durationStatistics, List createReader(k, v, topicCounters); activeTopics.add(k); durationStatistics.addAndGetThreads(1); - } catch (SQLException sqlException) { + } + catch (SQLException sqlException) { LOGGER.error("Topic <{}> not activated due to reader creation error: ", k, sqlException); } }); diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReader.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReader.java index f34f2dde..17febbe4 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReader.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReader.java @@ -43,7 +43,6 @@ * Teragrep, the applicable Commercial License may apply to this file if you as * a licensee so wish it. */ - package com.teragrep.cfe_39.consumers.kafka; import org.apache.kafka.clients.consumer.*; @@ -55,13 +54,16 @@ import java.util.*; public class KafkaReader implements AutoCloseable { + final Logger LOGGER = LoggerFactory.getLogger(KafkaReader.class); private Iterator> kafkaRecordsIterator = Collections.emptyIterator(); private final Consumer kafkaConsumer; private final java.util.function.Consumer> callbackFunction; public KafkaReader( - Consumer kafkaConsumer, java.util.function.Consumer> callbackFunction) { + Consumer kafkaConsumer, + java.util.function.Consumer> callbackFunction + ) { this.kafkaConsumer = kafkaConsumer; this.callbackFunction = callbackFunction; } @@ -81,7 +83,8 @@ public void read() { while (kafkaRecordsIterator.hasNext()) { ConsumerRecord record = kafkaRecordsIterator.next(); LOGGER.debug("adding from offset: <{}>", record.offset()); - recordOffsetObjectList.add(new RecordOffset(record.topic(), record.partition(), record.offset(), record.value())); + recordOffsetObjectList + .add(new RecordOffset(record.topic(), record.partition(), record.offset(), record.value())); } if (!recordOffsetObjectList.isEmpty()) { diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java index 5e346aa4..f7a9d81a 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java @@ -43,7 +43,6 @@ * Teragrep, the applicable Commercial License may apply to this file if you as * a licensee so wish it. */ - package com.teragrep.cfe_39.consumers.kafka; import org.apache.kafka.clients.consumer.Consumer; @@ -59,119 +58,174 @@ import java.util.*; /** - *

Mock Kafka Consumer Factory

- * - * Mocked Kafka Consumer factory used for testing. + *

Mock Kafka Consumer Factory

Mocked Kafka Consumer factory used for testing. * * @since 08/06/2022 * @author Mikko Kortelainen */ public class MockKafkaConsumerFactoryTemp { + final static private Logger LOGGER = LoggerFactory.getLogger(MockKafkaConsumerFactoryTemp.class); private MockKafkaConsumerFactoryTemp() { } private static void generateEvents(MockConsumer consumer, String topicName, int partition) { - consumer.addRecord(new ConsumerRecord<>(topicName, - partition, - 0L, - "2022-04-25T07:34:50.804Z".getBytes(StandardCharsets.UTF_8), - "<12>1 2022-04-25T07:34:50.804Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"835bf792-91cf-44e3-976b-518330bb8fd3\" source=\"source\" unixtime=\"1650872090805\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872090\"] [WARN] 2022-04-25 07:34:50,804 com.teragrep.jla_02.Log4j Log - Log4j warn says hi!".getBytes(StandardCharsets.UTF_8) - ) - ); - consumer.addRecord(new ConsumerRecord<>(topicName, - partition, - 1L, - "2022-04-25T07:34:50.806Z".getBytes(StandardCharsets.UTF_8), - "<12>1 2022-04-25T07:34:50.806Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"c3f13f9a-05e2-41bd-b0ad-1eca6fd6fd9a\" source=\"source\" unixtime=\"1650872090806\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872090\"] [ERROR] 2022-04-25 07:34:50,806 com.teragrep.jla_02.Log4j Log - Log4j error says hi!".getBytes(StandardCharsets.UTF_8) - ) - ); - consumer.addRecord(new ConsumerRecord<>(topicName, - partition, - 2L, - "2022-04-25T07:34:50.822Z".getBytes(StandardCharsets.UTF_8), - "<12>1 2022-04-25T07:34:50.822Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02\"][event_id@48577 hostname=\"jla-02\" uuid=\"1848d8a1-2f08-4a1e-bec4-ff9e6dd92553\" source=\"source\" unixtime=\"1650872090822\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872090\"] 470647 [Thread-3] INFO com.teragrep.jla_02.Logback Daily - Logback-daily says hi.".getBytes(StandardCharsets.UTF_8) - ) - ); - consumer.addRecord(new ConsumerRecord<>(topicName, - partition, - 3L, - "2022-04-25T07:34:50.822Z".getBytes(StandardCharsets.UTF_8), - "<12>1 2022-04-25T07:34:50.822Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02\"][event_id@48577 hostname=\"jla-02\" uuid=\"5e1a0398-c2a0-468d-a562-c3bb31f0f853\" source=\"source\" unixtime=\"1650872090822\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872090\"] 470646 [Thread-3] INFO com.teragrep.jla_02.Logback Audit - Logback-audit says hi.".getBytes(StandardCharsets.UTF_8) - ) - ); - consumer.addRecord(new ConsumerRecord<>(topicName, - partition, - 4L, - "2022-04-25T07:34:50.822Z".getBytes(StandardCharsets.UTF_8), - "<12>1 2022-04-25T07:34:50.822Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02\"][event_id@48577 hostname=\"jla-02\" uuid=\"6268c3a2-5bda-427f-acce-29416eb445f4\" source=\"source\" unixtime=\"1650872090822\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872090\"] 470647 [Thread-3] INFO com.teragrep.jla_02.Logback Metric - Logback-metric says hi.".getBytes(StandardCharsets.UTF_8) - ) - ); - consumer.addRecord(new ConsumerRecord<>(topicName, - partition, - 5L, - "2022-04-25T07:34:52.238Z".getBytes(StandardCharsets.UTF_8), - "<12>1 2022-04-25T07:34:52.238Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"b500dcaf-1101-4000-b6b9-bfb052ddbf86\" source=\"source\" unixtime=\"1650872092238\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.238 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info audit says hi!]".getBytes(StandardCharsets.UTF_8) - ) - ); - consumer.addRecord(new ConsumerRecord<>(topicName, - partition, - 6L, - "2022-04-25T07:34:52.239Z".getBytes(StandardCharsets.UTF_8), - "<12>1 2022-04-25T07:34:52.239Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"05363122-51ac-4c0b-a681-f5868081f56d\" source=\"source\" unixtime=\"1650872092239\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info daily says hi!]".getBytes(StandardCharsets.UTF_8) - ) - ); + consumer + .addRecord( + new ConsumerRecord<>( + topicName, + partition, + 0L, + "2022-04-25T07:34:50.804Z".getBytes(StandardCharsets.UTF_8), + "<12>1 2022-04-25T07:34:50.804Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"835bf792-91cf-44e3-976b-518330bb8fd3\" source=\"source\" unixtime=\"1650872090805\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872090\"] [WARN] 2022-04-25 07:34:50,804 com.teragrep.jla_02.Log4j Log - Log4j warn says hi!" + .getBytes(StandardCharsets.UTF_8) + ) + ); + consumer + .addRecord( + new ConsumerRecord<>( + topicName, + partition, + 1L, + "2022-04-25T07:34:50.806Z".getBytes(StandardCharsets.UTF_8), + "<12>1 2022-04-25T07:34:50.806Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"c3f13f9a-05e2-41bd-b0ad-1eca6fd6fd9a\" source=\"source\" unixtime=\"1650872090806\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872090\"] [ERROR] 2022-04-25 07:34:50,806 com.teragrep.jla_02.Log4j Log - Log4j error says hi!" + .getBytes(StandardCharsets.UTF_8) + ) + ); + consumer + .addRecord( + new ConsumerRecord<>( + topicName, + partition, + 2L, + "2022-04-25T07:34:50.822Z".getBytes(StandardCharsets.UTF_8), + "<12>1 2022-04-25T07:34:50.822Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02\"][event_id@48577 hostname=\"jla-02\" uuid=\"1848d8a1-2f08-4a1e-bec4-ff9e6dd92553\" source=\"source\" unixtime=\"1650872090822\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872090\"] 470647 [Thread-3] INFO com.teragrep.jla_02.Logback Daily - Logback-daily says hi." + .getBytes(StandardCharsets.UTF_8) + ) + ); + consumer + .addRecord( + new ConsumerRecord<>( + topicName, + partition, + 3L, + "2022-04-25T07:34:50.822Z".getBytes(StandardCharsets.UTF_8), + "<12>1 2022-04-25T07:34:50.822Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02\"][event_id@48577 hostname=\"jla-02\" uuid=\"5e1a0398-c2a0-468d-a562-c3bb31f0f853\" source=\"source\" unixtime=\"1650872090822\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872090\"] 470646 [Thread-3] INFO com.teragrep.jla_02.Logback Audit - Logback-audit says hi." + .getBytes(StandardCharsets.UTF_8) + ) + ); + consumer + .addRecord( + new ConsumerRecord<>( + topicName, + partition, + 4L, + "2022-04-25T07:34:50.822Z".getBytes(StandardCharsets.UTF_8), + "<12>1 2022-04-25T07:34:50.822Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02\"][event_id@48577 hostname=\"jla-02\" uuid=\"6268c3a2-5bda-427f-acce-29416eb445f4\" source=\"source\" unixtime=\"1650872090822\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872090\"] 470647 [Thread-3] INFO com.teragrep.jla_02.Logback Metric - Logback-metric says hi." + .getBytes(StandardCharsets.UTF_8) + ) + ); + consumer + .addRecord( + new ConsumerRecord<>( + topicName, + partition, + 5L, + "2022-04-25T07:34:52.238Z".getBytes(StandardCharsets.UTF_8), + "<12>1 2022-04-25T07:34:52.238Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"b500dcaf-1101-4000-b6b9-bfb052ddbf86\" source=\"source\" unixtime=\"1650872092238\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.238 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info audit says hi!]" + .getBytes(StandardCharsets.UTF_8) + ) + ); + consumer + .addRecord( + new ConsumerRecord<>( + topicName, + partition, + 6L, + "2022-04-25T07:34:52.239Z".getBytes(StandardCharsets.UTF_8), + "<12>1 2022-04-25T07:34:52.239Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"05363122-51ac-4c0b-a681-f5868081f56d\" source=\"source\" unixtime=\"1650872092239\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info daily says hi!]" + .getBytes(StandardCharsets.UTF_8) + ) + ); - consumer.addRecord(new ConsumerRecord<>(topicName, - partition, - 7L, - "2022-04-25T07:34:52.239Z".getBytes(StandardCharsets.UTF_8), - "<12>1 2022-04-25T07:34:52.239Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"7bbcd843-b795-4c14-b4a1-95f5d445cbcd\" source=\"source\" unixtime=\"1650872092239\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info metric says hi!]".getBytes(StandardCharsets.UTF_8) - ) - ); - consumer.addRecord(new ConsumerRecord<>(topicName, - partition, - 8L, - "2022-04-25T07:34:52.240Z".getBytes(StandardCharsets.UTF_8), - "<12>1 2022-04-25T07:34:52.240Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"2bc0a9f9-237d-4656-b40a-3038aace37f0\" source=\"source\" unixtime=\"1650872092240\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn audit says hi!]".getBytes(StandardCharsets.UTF_8) - ) - ); - consumer.addRecord(new ConsumerRecord<>(topicName, - partition, - 9L, - "2022-04-25T07:34:52.240Z".getBytes(StandardCharsets.UTF_8), - "<12>1 2022-04-25T07:34:52.240Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"ecf61e8d-e3a7-48ef-9b73-3c5a5243d2e6\" source=\"source\" unixtime=\"1650872092240\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn daily says hi!]".getBytes(StandardCharsets.UTF_8) - ) - ); - consumer.addRecord(new ConsumerRecord<>(topicName, - partition, - 10L, - "2022-04-25T07:34:52.241Z".getBytes(StandardCharsets.UTF_8), - "<12>1 2022-04-25T07:34:52.241Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"bf101d5a-e816-4f51-b132-97f8e3431f8e\" source=\"source\" unixtime=\"1650872092241\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.241 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn metric says hi!]".getBytes(StandardCharsets.UTF_8) - ) - ); - consumer.addRecord(new ConsumerRecord<>(topicName, - partition, - 11L, - "2022-04-25T07:34:52.241Z".getBytes(StandardCharsets.UTF_8), - "<12>1 2022-04-25T07:34:52.241Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"ef94d9e9-3c44-4892-b5a6-bf361d13ff97\" source=\"source\" unixtime=\"1650872092241\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.241 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error audit says hi!]".getBytes(StandardCharsets.UTF_8) - ) - ); - consumer.addRecord(new ConsumerRecord<>(topicName, - partition, - 12L, - "2022-04-25T07:34:52.242Z".getBytes(StandardCharsets.UTF_8), - "<12>1 2022-04-25T07:34:52.242Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"5bce6e3d-767d-44b4-a044-6c4872f8f2b5\" source=\"source\" unixtime=\"1650872092242\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.242 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error daily says hi!]".getBytes(StandardCharsets.UTF_8) - ) - ); - consumer.addRecord(new ConsumerRecord<>(topicName, - partition, - 13L, - "2022-04-25T07:34:52.243Z".getBytes(StandardCharsets.UTF_8), - "<12>1 2022-04-25T07:34:52.243Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"3bb55ce4-0ea7-413a-b403-28b174d7ac99\" source=\"source\" unixtime=\"1650872092243\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.243 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error metric says hi!]".getBytes(StandardCharsets.UTF_8) - ) - ); + consumer + .addRecord( + new ConsumerRecord<>( + topicName, + partition, + 7L, + "2022-04-25T07:34:52.239Z".getBytes(StandardCharsets.UTF_8), + "<12>1 2022-04-25T07:34:52.239Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"7bbcd843-b795-4c14-b4a1-95f5d445cbcd\" source=\"source\" unixtime=\"1650872092239\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info metric says hi!]" + .getBytes(StandardCharsets.UTF_8) + ) + ); + consumer + .addRecord( + new ConsumerRecord<>( + topicName, + partition, + 8L, + "2022-04-25T07:34:52.240Z".getBytes(StandardCharsets.UTF_8), + "<12>1 2022-04-25T07:34:52.240Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"2bc0a9f9-237d-4656-b40a-3038aace37f0\" source=\"source\" unixtime=\"1650872092240\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn audit says hi!]" + .getBytes(StandardCharsets.UTF_8) + ) + ); + consumer + .addRecord( + new ConsumerRecord<>( + topicName, + partition, + 9L, + "2022-04-25T07:34:52.240Z".getBytes(StandardCharsets.UTF_8), + "<12>1 2022-04-25T07:34:52.240Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"ecf61e8d-e3a7-48ef-9b73-3c5a5243d2e6\" source=\"source\" unixtime=\"1650872092240\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn daily says hi!]" + .getBytes(StandardCharsets.UTF_8) + ) + ); + consumer + .addRecord( + new ConsumerRecord<>( + topicName, + partition, + 10L, + "2022-04-25T07:34:52.241Z".getBytes(StandardCharsets.UTF_8), + "<12>1 2022-04-25T07:34:52.241Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"bf101d5a-e816-4f51-b132-97f8e3431f8e\" source=\"source\" unixtime=\"1650872092241\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.241 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn metric says hi!]" + .getBytes(StandardCharsets.UTF_8) + ) + ); + consumer + .addRecord( + new ConsumerRecord<>( + topicName, + partition, + 11L, + "2022-04-25T07:34:52.241Z".getBytes(StandardCharsets.UTF_8), + "<12>1 2022-04-25T07:34:52.241Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"ef94d9e9-3c44-4892-b5a6-bf361d13ff97\" source=\"source\" unixtime=\"1650872092241\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.241 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error audit says hi!]" + .getBytes(StandardCharsets.UTF_8) + ) + ); + consumer + .addRecord( + new ConsumerRecord<>( + topicName, + partition, + 12L, + "2022-04-25T07:34:52.242Z".getBytes(StandardCharsets.UTF_8), + "<12>1 2022-04-25T07:34:52.242Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"5bce6e3d-767d-44b4-a044-6c4872f8f2b5\" source=\"source\" unixtime=\"1650872092242\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.242 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error daily says hi!]" + .getBytes(StandardCharsets.UTF_8) + ) + ); + consumer + .addRecord( + new ConsumerRecord<>( + topicName, + partition, + 13L, + "2022-04-25T07:34:52.243Z".getBytes(StandardCharsets.UTF_8), + "<12>1 2022-04-25T07:34:52.243Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"3bb55ce4-0ea7-413a-b403-28b174d7ac99\" source=\"source\" unixtime=\"1650872092243\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.243 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error metric says hi!]" + .getBytes(StandardCharsets.UTF_8) + ) + ); } // Can initialize topic scan with all partitions available when the input parameter is 0. Consumer is manually assigned to specific partitions depending on the threadnum parameter. For example on threadnum 1 consumer has odd numbered partitions assigned to it and threadnum 2 has the even numbered partitions. @@ -197,30 +251,32 @@ public static Consumer getConsumer(int threadnum) { if (threadnum == 1) { List oddTopicPartitions = new ArrayList<>(); for (TopicPartition a : topicPartitions) { - if(((a.partition() + 1) % 2) == 0) { + if (((a.partition() + 1) % 2) == 0) { oddTopicPartitions.add(a); } } consumer.assign(oddTopicPartitions); // assign for (TopicPartition a : topicPartitions) { - if(((a.partition() + 1) % 2) == 0) { + if (((a.partition() + 1) % 2) == 0) { generateEvents(consumer, a.topic(), a.partition()); } } - } else if (threadnum == 2) { + } + else if (threadnum == 2) { List evenTopicPartitions = new ArrayList<>(); for (TopicPartition a : topicPartitions) { - if(((a.partition() + 1) % 2) != 0) { + if (((a.partition() + 1) % 2) != 0) { evenTopicPartitions.add(a); } } consumer.assign(evenTopicPartitions); // assign for (TopicPartition a : topicPartitions) { - if(((a.partition() + 1) % 2) != 0) { + if (((a.partition() + 1) % 2) != 0) { generateEvents(consumer, a.topic(), a.partition()); } } - }else { + } + else { consumer.assign(topicPartitions); // assign for (TopicPartition a : topicPartitions) { generateEvents(consumer, a.topic(), a.partition()); @@ -233,4 +289,4 @@ public static Consumer getConsumer(int threadnum) { consumer.updatePartitions("testConsumerTopic", mockPartitionInfo); return consumer; } -} \ No newline at end of file +} diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/NullOffset.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/NullOffset.java index 73cee40e..235a7ff4 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/NullOffset.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/NullOffset.java @@ -43,10 +43,9 @@ * Teragrep, the applicable Commercial License may apply to this file if you as * a licensee so wish it. */ - package com.teragrep.cfe_39.consumers.kafka; -public class NullOffset extends Offset{ +public class NullOffset extends Offset { @Override public boolean isNull() { diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/Offset.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/Offset.java index 80474527..8dd194e5 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/Offset.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/Offset.java @@ -43,17 +43,22 @@ * Teragrep, the applicable Commercial License may apply to this file if you as * a licensee so wish it. */ - package com.teragrep.cfe_39.consumers.kafka; public abstract class Offset { + protected String topic; protected Integer partition; protected Long offset; protected byte[] record; + public abstract boolean isNull(); + public abstract String getTopic(); + public abstract Integer getPartition(); + public abstract Long getOffset(); + public abstract byte[] getRecord(); } diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinator.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinator.java index 0f29cb24..fb9f8d8e 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinator.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinator.java @@ -43,7 +43,6 @@ * Teragrep, the applicable Commercial License may apply to this file if you as * a licensee so wish it. */ - package com.teragrep.cfe_39.consumers.kafka; import org.apache.kafka.clients.consumer.KafkaConsumer; @@ -56,8 +55,8 @@ import java.util.function.Consumer; public class ReadCoordinator implements Runnable { - private static final Logger LOGGER = - LoggerFactory.getLogger(ReadCoordinator.class); + + private static final Logger LOGGER = LoggerFactory.getLogger(ReadCoordinator.class); private final String queueTopic; private final Properties readerKafkaProperties; @@ -69,29 +68,37 @@ public ReadCoordinator( String queueTopic, Properties readerKafkaProperties, Consumer> callbackFunction, - Map hdfsStartOffsets) - { + Map hdfsStartOffsets + ) { this.queueTopic = queueTopic; this.readerKafkaProperties = readerKafkaProperties; this.callbackFunction = callbackFunction; this.hdfsStartOffsets = hdfsStartOffsets; } - private KafkaReader createKafkaReader(Properties readerKafkaProperties, - String topic, - Consumer> callbackFunction, - boolean useMockKafkaConsumer) { + private KafkaReader createKafkaReader( + Properties readerKafkaProperties, + String topic, + Consumer> callbackFunction, + boolean useMockKafkaConsumer + ) { org.apache.kafka.clients.consumer.Consumer kafkaConsumer; if (useMockKafkaConsumer) { // Mock kafka consumer is enabled, create mock consumers with assigned partitions that are not overlapping with each other. String name = Thread.currentThread().getName(); // Use thread name to identify which thread is running the code. if (Objects.equals(name, "testConsumerTopic1")) { kafkaConsumer = MockKafkaConsumerFactoryTemp.getConsumer(1); // creates a Kafka MockConsumer that has the odd numbered partitions assigned to it. - }else { + } + else { kafkaConsumer = MockKafkaConsumerFactoryTemp.getConsumer(2); // creates a Kafka MockConsumer that has the even numbered partitions assigned to it. } - } else { // Mock kafka consumer is disabled, subscribe method should handle assigning the partitions automatically to the consumer based on group id parameters of readerKafkaProperties. - kafkaConsumer = new KafkaConsumer<>(readerKafkaProperties, new ByteArrayDeserializer(), new ByteArrayDeserializer()); + } + else { // Mock kafka consumer is disabled, subscribe method should handle assigning the partitions automatically to the consumer based on group id parameters of readerKafkaProperties. + kafkaConsumer = new KafkaConsumer<>( + readerKafkaProperties, + new ByteArrayDeserializer(), + new ByteArrayDeserializer() + ); kafkaConsumer.subscribe(Collections.singletonList(topic)); } @@ -112,13 +119,11 @@ private KafkaReader createKafkaReader(Properties readerKafkaProperties, // Part or Runnable implementation, called when the thread is started. @Override public void run() { - boolean useMockKafkaConsumer = Boolean.parseBoolean(readerKafkaProperties.getProperty("useMockKafkaConsumer", "false")); + boolean useMockKafkaConsumer = Boolean + .parseBoolean(readerKafkaProperties.getProperty("useMockKafkaConsumer", "false")); try ( KafkaReader kafkaReader = createKafkaReader( - readerKafkaProperties, - queueTopic, - callbackFunction, - useMockKafkaConsumer + readerKafkaProperties, queueTopic, callbackFunction, useMockKafkaConsumer ) ) { while (run) { @@ -131,4 +136,4 @@ public void run() { public void stop() { run = false; } -} \ No newline at end of file +} diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/RecordOffset.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/RecordOffset.java index 216f7c97..72332842 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/RecordOffset.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/RecordOffset.java @@ -43,18 +43,12 @@ * Teragrep, the applicable Commercial License may apply to this file if you as * a licensee so wish it. */ - package com.teragrep.cfe_39.consumers.kafka; // This is the class for handling the Kafka record topic/partition/offset data that are required for HDFS storage. -public class RecordOffset extends Offset{ +public class RecordOffset extends Offset { - public RecordOffset( - String topic, - int partition, - long offset, - byte[] record - ) { + public RecordOffset(String topic, int partition, long offset, byte[] record) { this.topic = topic; this.partition = partition; this.offset = offset; diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java index 248c608a..20a14e87 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java @@ -43,7 +43,6 @@ * Teragrep, the applicable Commercial License may apply to this file if you as * a licensee so wish it. */ - package com.teragrep.cfe_39.consumers.kafka; import com.teragrep.cfe_39.avro.SyslogRecord; @@ -57,14 +56,12 @@ import org.slf4j.LoggerFactory; import java.io.*; -import java.nio.ByteBuffer; class SyslogAvroWriter implements AutoCloseable { private static final Logger LOGGER = LoggerFactory.getLogger(SyslogAvroWriter.class); - private final DatumWriter datumWriter = - new SpecificDatumWriter<>(SyslogRecord.class); + private final DatumWriter datumWriter = new SpecificDatumWriter<>(SyslogRecord.class); private final SyncableFileOutputStream syncableFileOutputStream; @@ -73,9 +70,7 @@ class SyslogAvroWriter implements AutoCloseable { SyslogAvroWriter(File syslogFile) throws IOException { dataFileWriter.setCodec(CodecFactory.snappyCodec()); - - syncableFileOutputStream = - new SyncableFileOutputStream(syslogFile); + syncableFileOutputStream = new SyncableFileOutputStream(syslogFile); syncableFileOutputStream.getChannel().tryLock(); @@ -93,8 +88,7 @@ class SyslogAvroWriter implements AutoCloseable { } } - - void write(SyslogRecord syslogRecord) throws IOException{ + void write(SyslogRecord syslogRecord) throws IOException { dataFileWriter.append(syslogRecord); dataFileWriter.flush(); // getFileSize() doesn't work properly if dataFileWriter.flush() is not called after appending a new record to the AVRO-file. @@ -107,4 +101,4 @@ public void close() throws IOException { public long getFileSize() throws IOException { return syncableFileOutputStream.getChannel().size(); } -} \ No newline at end of file +} diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/queue/QueueUtilities.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/queue/QueueUtilities.java index 887672e4..834ebbe0 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/queue/QueueUtilities.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/queue/QueueUtilities.java @@ -43,7 +43,6 @@ * Teragrep, the applicable Commercial License may apply to this file if you as * a licensee so wish it. */ - package com.teragrep.cfe_39.consumers.kafka.queue; import org.slf4j.Logger; @@ -56,18 +55,21 @@ import java.util.function.ToLongFunction; class QueueUtilities { - private static final Logger LOGGER = - LoggerFactory.getLogger(QueueUtilities.class); + + private static final Logger LOGGER = LoggerFactory.getLogger(QueueUtilities.class); static BiPredicate getFileMatcher(String queueNamePrefix) { return (path, basicFileAttributes) -> { if (!path.getFileName().toString().startsWith(queueNamePrefix)) { return false; - } else if (path.getFileName().toString().endsWith(".state")) { + } + else if (path.getFileName().toString().endsWith(".state")) { return false; - } else if (!basicFileAttributes.isRegularFile()) { + } + else if (!basicFileAttributes.isRegularFile()) { return false; - } else { + } + else { LOGGER.trace("getFileMatcher returning: <{}>", path); return true; } @@ -76,15 +78,11 @@ static BiPredicate getFileMatcher(String queueNamePre static void accessCheck(Path queueDirectory) { if (!Files.isDirectory(queueDirectory)) { - throw new IllegalArgumentException( - "Provided path is not a " - + "directory <[" + queueDirectory + "]>"); + throw new IllegalArgumentException("Provided path is not a " + "directory <[" + queueDirectory + "]>"); } if (!Files.isWritable(queueDirectory)) { - throw new IllegalArgumentException( - "Provided path is not " - + "writeable <[" + queueDirectory + "]>"); + throw new IllegalArgumentException("Provided path is not " + "writeable <[" + queueDirectory + "]>"); } } @@ -99,4 +97,4 @@ static ToLongFunction getPathToSequenceNumberFunction() { return Long.parseLong(sequenceNumberString); }; } -} \ No newline at end of file +} diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/queue/WritableQueue.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/queue/WritableQueue.java index 0ae38766..3df56a93 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/queue/WritableQueue.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/queue/WritableQueue.java @@ -43,7 +43,6 @@ * Teragrep, the applicable Commercial License may apply to this file if you as * a licensee so wish it. */ - package com.teragrep.cfe_39.consumers.kafka.queue; import java.io.File; @@ -56,12 +55,11 @@ import java.util.stream.Stream; public class WritableQueue { + private final Path queueDirectory; private String queueNamePrefix; - public WritableQueue( - String queueDirectory - ) { + public WritableQueue(String queueDirectory) { this.queueDirectory = Paths.get(queueDirectory); this.queueNamePrefix = ""; QueueUtilities.accessCheck(this.queueDirectory); @@ -69,26 +67,17 @@ public WritableQueue( private File getNextWritableFilename() throws IOException { - try (Stream files = Files.find( - queueDirectory, - 1, - QueueUtilities.getFileMatcher(queueNamePrefix), - FileVisitOption.FOLLOW_LINKS - )) { + try ( + Stream files = Files.find(queueDirectory, 1, QueueUtilities.getFileMatcher(queueNamePrefix), FileVisitOption.FOLLOW_LINKS) + ) { - long sequenceNumber = files.mapToLong( - QueueUtilities.getPathToSequenceNumberFunction() - ).max().orElse(0); + long sequenceNumber = files.mapToLong(QueueUtilities.getPathToSequenceNumberFunction()).max().orElse(0); long nextSequenceNumber = sequenceNumber + 1; // create next return new File( - queueDirectory.toAbsolutePath() - + File.separator - + queueNamePrefix - + "." - + nextSequenceNumber + queueDirectory.toAbsolutePath() + File.separator + queueNamePrefix + "." + nextSequenceNumber ); } catch (UncheckedIOException uncheckedIOException) { @@ -98,9 +87,10 @@ private File getNextWritableFilename() throws IOException { } public File getNextWritableFile() throws IOException { - if (queueNamePrefix.isEmpty()){ + if (queueNamePrefix.isEmpty()) { throw new IOException("No queueNamePrefix set"); - }else { + } + else { return getNextWritableFilename(); } } @@ -108,4 +98,4 @@ public File getNextWritableFile() throws IOException { public void setQueueNamePrefix(String a) { this.queueNamePrefix = a; } -} \ No newline at end of file +} diff --git a/src/main/java/com/teragrep/cfe_39/metrics/DurationStatistics.java b/src/main/java/com/teragrep/cfe_39/metrics/DurationStatistics.java index ac32fcd6..15454eaf 100644 --- a/src/main/java/com/teragrep/cfe_39/metrics/DurationStatistics.java +++ b/src/main/java/com/teragrep/cfe_39/metrics/DurationStatistics.java @@ -43,7 +43,6 @@ * Teragrep, the applicable Commercial License may apply to this file if you as * a licensee so wish it. */ - package com.teragrep.cfe_39.metrics; import com.codahale.metrics.Counter; @@ -55,6 +54,7 @@ import java.time.Instant; public class DurationStatistics { + MetricRegistry metricRegistry = new MetricRegistry(); private static final Logger LOGGER = LoggerFactory.getLogger(DurationStatistics.class); private Instant lastReportTime = Instant.now(); @@ -89,13 +89,15 @@ public void report() { // Check if new records were processed if (currentRecords > lastRecords) { records.inc(currentRecords - lastRecords); // new records found, adding the number of records to records. - }else { + } + else { long current = records.getCount(); records.dec(current); // no new records so set the counter back to 0. } if (currentBytes > lastBytes) { bytes.inc(currentBytes - lastBytes); // new records found, adding the number of records to records. - }else { + } + else { long current = bytes.getCount(); bytes.dec(current); // no new records so set the counter back to 0. } @@ -104,8 +106,8 @@ public void report() { long took = currentTime.toEpochMilli() - lastReportTime.toEpochMilli(); samplingIntervalStat.inc(took); - recordsPerSecondStat.mark(currentRecords-lastRecords); - bytesPerSecondStat.mark(currentBytes-lastBytes); + recordsPerSecondStat.mark(currentRecords - lastRecords); + bytesPerSecondStat.mark(currentBytes - lastBytes); // persist lastReportTime = currentTime; @@ -118,9 +120,15 @@ public long getTotalRecords() { } public void log() { - LOGGER.info("## Processed records <{}> and size <{}> KB during <{}> ms / Metrics for the preceding minute: <{}> RPS. <{}> KB/s ", records.getCount(), bytes.getCount() / 1024, samplingIntervalStat.getCount(), recordsPerSecondStat.getOneMinuteRate(), bytesPerSecondStat.getOneMinuteRate() / 1024); + LOGGER + .info( + "## Processed records <{}> and size <{}> KB during <{}> ms / Metrics for the preceding minute: <{}> RPS. <{}> KB/s ", + records.getCount(), bytes.getCount() / 1024, samplingIntervalStat.getCount(), + recordsPerSecondStat.getOneMinuteRate(), bytesPerSecondStat.getOneMinuteRate() / 1024 + ); samplingIntervalStat.dec(samplingIntervalStat.getCount()); } + public long addAndGetThreads(long delta) { threadsStat.mark(delta); return threadsStat.getCount(); diff --git a/src/main/java/com/teragrep/cfe_39/metrics/topic/TopicCounter.java b/src/main/java/com/teragrep/cfe_39/metrics/topic/TopicCounter.java index 132925c0..cd08a71c 100644 --- a/src/main/java/com/teragrep/cfe_39/metrics/topic/TopicCounter.java +++ b/src/main/java/com/teragrep/cfe_39/metrics/topic/TopicCounter.java @@ -43,13 +43,12 @@ * Teragrep, the applicable Commercial License may apply to this file if you as * a licensee so wish it. */ - package com.teragrep.cfe_39.metrics.topic; - import java.util.concurrent.atomic.AtomicLong; public class TopicCounter { + private final String topicName; private final AtomicLong totalRecords = new AtomicLong(); private final AtomicLong totalBytes = new AtomicLong(); @@ -61,11 +60,12 @@ public class TopicCounter { public TopicCounter(String topicName) { this.topicName = topicName; } + public long getTotalRecords() { return totalRecords.get(); } - public long getTotalBytes () { + public long getTotalBytes() { return totalBytes.get(); } diff --git a/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java b/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java index 172061b0..704211cf 100644 --- a/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java +++ b/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java @@ -43,7 +43,6 @@ * Teragrep, the applicable Commercial License may apply to this file if you as * a licensee so wish it. */ - package com.teragrep.cfe_39; import com.teragrep.cfe_39.avro.SyslogRecord; @@ -66,9 +65,7 @@ import java.io.IOException; import java.net.URI; import java.nio.file.Files; -import java.sql.Timestamp; import java.util.*; -import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -79,17 +76,18 @@ public class CombinedFullTest { private static File baseDir; private static Config config; - // Start minicluster and initialize config. @BeforeAll public static void startMiniCluster() throws IOException, InterruptedException { config = null; try { config = new Config(); - } catch (IOException e){ + } + catch (IOException e) { LOGGER.error("Can't load config: {}", e.toString()); Assertions.fail(); - } catch (IllegalArgumentException e) { + } + catch (IllegalArgumentException e) { LOGGER.error("Got invalid config: {}", e.toString()); Assertions.fail(); } @@ -99,7 +97,7 @@ public static void startMiniCluster() throws IOException, InterruptedException { conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, baseDir.getAbsolutePath()); MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf); hdfsCluster = builder.build(); - String hdfsURI = "hdfs://localhost:"+ hdfsCluster.getNameNodePort() + "/"; + String hdfsURI = "hdfs://localhost:" + hdfsCluster.getNameNodePort() + "/"; config.setHdfsuri(hdfsURI); DistributedFileSystem fileSystem = hdfsCluster.getFileSystem(); } @@ -124,18 +122,20 @@ public void kafkaAndAvroFullTest() throws InterruptedException, IOException { if (config.getPruneOffset() == 157784760000L) { try { hdfsReadCheck(); - } catch (IOException e) { + } + catch (IOException e) { throw new RuntimeException(e); } } } + @Test public void hdfsPruneTest() throws IOException { // Check that the files were properly written to HDFS with a read test. String hdfsuri = config.getHdfsuri(); - String path = config.getHdfsPath()+"/"+"testConsumerTopic"; + String path = config.getHdfsPath() + "/" + "testConsumerTopic"; // ====== Init HDFS File System Object Configuration conf = new Configuration(); // Set FileSystem URI @@ -150,22 +150,22 @@ public void hdfsPruneTest() throws IOException { FileSystem fs = FileSystem.get(URI.create(hdfsuri), conf); //==== Create directory if not exists - Path workingDir=fs.getWorkingDirectory(); - Path newDirectoryPath= new Path(path); - if(!fs.exists(newDirectoryPath)) { + Path workingDir = fs.getWorkingDirectory(); + Path newDirectoryPath = new Path(path); + if (!fs.exists(newDirectoryPath)) { // Create new Directory fs.mkdirs(newDirectoryPath); LOGGER.info("Path {} created.", path); } /* Use either HDFS-file modification timestamps or avro-mapred for pruning. - + The records are in this AVRO format: {"timestamp": 1650872092240000, "message": "25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn audit says hi!]", "directory": "jla02logger", "stream": "test:jla02logger:0", "host": "jla-02.default", "input": "imrelp:cfe-06-0.cfe-06.default:", "partition": "8", "offset": 8, "origin": "jla-02.default"} Query handler must be implemented in a way that the AVRO files are first opened, then processed to syslog format and then sent to the query requester. The records are processed/filtered based on the given query conditions using MapReduce to make the code capable of processing the vast amounts of records that are expected. MapReduce functionalities of the Hadoop cluster: https://hadoop.apache.org/docs/stable/hadoop-mapreduce-client/hadoop-mapreduce-client-core/MapReduceTutorial.html Avro side of documentations for MapReduce: https://avro.apache.org/docs/1.11.1/mapreduce-guide/ - + Another method for pruning aside using avro-mapred is to use modification timestamp of the avro-file stored in HDFS: fs.setTimes(new Path(path+"/"+0.8), Long.parseUnsignedLong("1675930598000"), -1); where mtime is modification time and atime is access time. -1 as input parameter leaves the original atime/mtime value as is.*/ @@ -186,7 +186,8 @@ public void hdfsPruneTest() throws IOException { } Assertions.assertTrue(delete); LOGGER.info("All files were pruned properly."); - }else { + } + else { if (config.getPruneOffset() == 157784760000L) { Assertions.fail("There were no files available in the database when there should be."); } @@ -200,7 +201,7 @@ public void hdfsReadCheck() throws IOException { // Check that the files were properly written to HDFS with a read test. String hdfsuri = config.getHdfsuri(); - String path = config.getHdfsPath()+"/"+"testConsumerTopic"; + String path = config.getHdfsPath() + "/" + "testConsumerTopic"; // ====== Init HDFS File System Object Configuration conf = new Configuration(); // Set FileSystem URI @@ -215,9 +216,9 @@ public void hdfsReadCheck() throws IOException { FileSystem fs = FileSystem.get(URI.create(hdfsuri), conf); //==== Create directory if not exists - Path workingDir=fs.getWorkingDirectory(); - Path newDirectoryPath= new Path(path); - if(!fs.exists(newDirectoryPath)) { + Path workingDir = fs.getWorkingDirectory(); + Path newDirectoryPath = new Path(path); + if (!fs.exists(newDirectoryPath)) { // Create new Directory fs.mkdirs(newDirectoryPath); LOGGER.info("Path {} created.", path); @@ -225,7 +226,7 @@ public void hdfsReadCheck() throws IOException { /* This is the HDFS write path for the files: Path hdfswritepath = new Path(newDirectoryPath + "/" + fileName); where newDirectoryPath is config.getHdfsPath() + "/" + lastObject.topic; and filename is lastObject.partition+"."+lastObject.offset; - + Create the list of files to read from HDFS. Test setup is created so each of the 0-9 partitions will have 2 files with offsets of 8 and 13.*/ List filenameList = new ArrayList<>(); for (int i = 0; i <= 9; i++) { @@ -242,7 +243,10 @@ public void hdfsReadCheck() throws IOException { //Init input stream FSDataInputStream inputStream = fs.open(hdfsreadpath); //The data is in AVRO-format, so it can't be read as a string. - DataFileStream reader = new DataFileStream<>(inputStream, new SpecificDatumReader<>(SyslogRecord.class)); + DataFileStream reader = new DataFileStream<>( + inputStream, + new SpecificDatumReader<>(SyslogRecord.class) + ); SyslogRecord record = null; LOGGER.info("\nReading records from file {}:", hdfsreadpath.toString()); while (reader.hasNext()) { @@ -250,46 +254,143 @@ record = reader.next(record); LOGGER.debug(record.toString()); // Assert records here like it is done in KafkaConsumerTest.avroReader(). if (looper <= 0) { - Assertions.assertEquals("{\"timestamp\": 1650872090804000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 0, \"origin\": \"jla-02.default\", \"payload\": \"[WARN] 2022-04-25 07:34:50,804 com.teragrep.jla_02.Log4j Log - Log4j warn says hi!\"}", record.toString()); + Assertions + .assertEquals( + "{\"timestamp\": 1650872090804000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 0, \"origin\": \"jla-02.default\", \"payload\": \"[WARN] 2022-04-25 07:34:50,804 com.teragrep.jla_02.Log4j Log - Log4j warn says hi!\"}", + record.toString() + ); looper++; - } else if (looper == 1) { - Assertions.assertEquals("{\"timestamp\": 1650872090806000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 1, \"origin\": \"jla-02.default\", \"payload\": \"[ERROR] 2022-04-25 07:34:50,806 com.teragrep.jla_02.Log4j Log - Log4j error says hi!\"}", record.toString()); + } + else if (looper == 1) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872090806000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 1, \"origin\": \"jla-02.default\", \"payload\": \"[ERROR] 2022-04-25 07:34:50,806 com.teragrep.jla_02.Log4j Log - Log4j error says hi!\"}", + record.toString() + ); looper++; - } else if (looper == 2) { - Assertions.assertEquals("{\"timestamp\": 1650872090822000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 2, \"origin\": \"jla-02\", \"payload\": \"470647 [Thread-3] INFO com.teragrep.jla_02.Logback Daily - Logback-daily says hi.\"}", record.toString()); + } + else if (looper == 2) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872090822000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 2, \"origin\": \"jla-02\", \"payload\": \"470647 [Thread-3] INFO com.teragrep.jla_02.Logback Daily - Logback-daily says hi.\"}", + record.toString() + ); looper++; - } else if (looper == 3) { - Assertions.assertEquals("{\"timestamp\": 1650872090822000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 3, \"origin\": \"jla-02\", \"payload\": \"470646 [Thread-3] INFO com.teragrep.jla_02.Logback Audit - Logback-audit says hi.\"}", record.toString()); + } + else if (looper == 3) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872090822000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 3, \"origin\": \"jla-02\", \"payload\": \"470646 [Thread-3] INFO com.teragrep.jla_02.Logback Audit - Logback-audit says hi.\"}", + record.toString() + ); looper++; - } else if (looper == 4) { - Assertions.assertEquals("{\"timestamp\": 1650872090822000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 4, \"origin\": \"jla-02\", \"payload\": \"470647 [Thread-3] INFO com.teragrep.jla_02.Logback Metric - Logback-metric says hi.\"}", record.toString()); + } + else if (looper == 4) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872090822000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 4, \"origin\": \"jla-02\", \"payload\": \"470647 [Thread-3] INFO com.teragrep.jla_02.Logback Metric - Logback-metric says hi.\"}", + record.toString() + ); looper++; - } else if (looper == 5) { - Assertions.assertEquals("{\"timestamp\": 1650872092238000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 5, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.238 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info audit says hi!]\"}", record.toString()); + } + else if (looper == 5) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872092238000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 5, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.238 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info audit says hi!]\"}", + record.toString() + ); looper++; - } else if (looper == 6) { - Assertions.assertEquals("{\"timestamp\": 1650872092239000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 6, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info daily says hi!]\"}", record.toString()); + } + else if (looper == 6) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872092239000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 6, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info daily says hi!]\"}", + record.toString() + ); looper++; - } else if (looper == 7) { - Assertions.assertEquals("{\"timestamp\": 1650872092239000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 7, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info metric says hi!]\"}", record.toString()); + } + else if (looper == 7) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872092239000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 7, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info metric says hi!]\"}", + record.toString() + ); looper++; - } else if (looper == 8) { - Assertions.assertEquals("{\"timestamp\": 1650872092240000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 8, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn audit says hi!]\"}", record.toString()); + } + else if (looper == 8) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872092240000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 8, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn audit says hi!]\"}", + record.toString() + ); looper++; - } else if (looper == 9) { - Assertions.assertEquals("{\"timestamp\": 1650872092240000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 9, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn daily says hi!]\"}", record.toString()); + } + else if (looper == 9) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872092240000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 9, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn daily says hi!]\"}", + record.toString() + ); looper++; - } else if (looper == 10) { - Assertions.assertEquals("{\"timestamp\": 1650872092241000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 10, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.241 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn metric says hi!]\"}", record.toString()); + } + else if (looper == 10) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872092241000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 10, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.241 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn metric says hi!]\"}", + record.toString() + ); looper++; - } else if (looper == 11) { - Assertions.assertEquals("{\"timestamp\": 1650872092241000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 11, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.241 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error audit says hi!]\"}", record.toString()); + } + else if (looper == 11) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872092241000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 11, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.241 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error audit says hi!]\"}", + record.toString() + ); looper++; - } else if (looper == 12) { - Assertions.assertEquals("{\"timestamp\": 1650872092242000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 12, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.242 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error daily says hi!]\"}", record.toString()); + } + else if (looper == 12) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872092242000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 12, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.242 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error daily says hi!]\"}", + record.toString() + ); looper++; - } else { - Assertions.assertEquals("{\"timestamp\": 1650872092243000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 13, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.243 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error metric says hi!]\"}", record.toString()); + } + else { + Assertions + .assertEquals( + "{\"timestamp\": 1650872092243000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 13, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.243 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error metric says hi!]\"}", + record.toString() + ); looper = 0; LOGGER.info("Partition {} passed assertions.", partitionCounter); partitionCounter++; @@ -301,7 +402,7 @@ record = reader.next(record); } private void insertMockFiles() throws IOException { - String path = config.getHdfsPath()+"/"+"testConsumerTopic"; // "hdfs:///opt/teragrep/cfe_39/srv/testConsumerTopic" + String path = config.getHdfsPath() + "/" + "testConsumerTopic"; // "hdfs:///opt/teragrep/cfe_39/srv/testConsumerTopic" String hdfsURI = config.getHdfsuri(); // ====== Init HDFS File System Object Configuration conf = new Configuration(); @@ -316,7 +417,6 @@ private void insertMockFiles() throws IOException { //Get the filesystem - HDFS FileSystem fs = FileSystem.get(URI.create(hdfsURI), conf); - //==== Create directory if not exists Path workingDir = fs.getWorkingDirectory(); // Sets the directory where the data should be stored, if the directory doesn't exist then it's created. @@ -327,8 +427,9 @@ private void insertMockFiles() throws IOException { LOGGER.debug("Path {} created.", path); } - String dir = System.getProperty("user.dir")+"/src/test/java/com/teragrep/cfe_39/mockHdfsFiles"; - Set listOfFiles = Stream.of(Objects.requireNonNull(new File(dir).listFiles())) + String dir = System.getProperty("user.dir") + "/src/test/java/com/teragrep/cfe_39/mockHdfsFiles"; + Set listOfFiles = Stream + .of(Objects.requireNonNull(new File(dir).listFiles())) .filter(file -> !file.isDirectory()) .map(File::getName) .collect(Collectors.toSet()); diff --git a/src/test/java/com/teragrep/cfe_39/HdfsTest.java b/src/test/java/com/teragrep/cfe_39/HdfsTest.java index 0f0582fc..1613025f 100644 --- a/src/test/java/com/teragrep/cfe_39/HdfsTest.java +++ b/src/test/java/com/teragrep/cfe_39/HdfsTest.java @@ -43,7 +43,6 @@ * Teragrep, the applicable Commercial License may apply to this file if you as * a licensee so wish it. */ - package com.teragrep.cfe_39; import com.teragrep.cfe_39.avro.SyslogRecord; @@ -71,6 +70,7 @@ import java.nio.file.Files; public class HdfsTest { + private static final Logger LOGGER = LoggerFactory.getLogger(HdfsTest.class); private static MiniDFSCluster hdfsCluster; @@ -83,10 +83,12 @@ public static void generateTestData() throws IOException, InterruptedException { config = null; try { config = new Config(); - } catch (IOException e){ + } + catch (IOException e) { LOGGER.error("Can't load config: " + e); System.exit(1); - } catch (IllegalArgumentException e) { + } + catch (IllegalArgumentException e) { LOGGER.error("Got invalid config: " + e); System.exit(1); } @@ -103,7 +105,7 @@ public static void startMiniCluster() throws IOException { conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, baseDir.getAbsolutePath()); MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf); hdfsCluster = builder.build(); - String hdfsURI = "hdfs://localhost:"+ hdfsCluster.getNameNodePort() + "/"; + String hdfsURI = "hdfs://localhost:" + hdfsCluster.getNameNodePort() + "/"; LOGGER.debug("hdfsURI: " + hdfsURI); config.setHdfsuri(hdfsURI); DistributedFileSystem fileSystem = hdfsCluster.getFileSystem(); @@ -115,17 +117,11 @@ public static void deleteTestData() throws IOException { Path queueDirectory = new Path(config.getQueueDirectory()); // Paths.get(config.getQueueDirectory()); for (int j = 0; j <= 9; j++) { for (int i = 1; i <= 2; i++) { - File syslogFile = new File( - queueDirectory.toUri() - + File.separator - + "testConsumerTopic" - + j - + "." - + i - ); + File syslogFile = new File(queueDirectory.toUri() + File.separator + "testConsumerTopic" + j + "." + i); try { boolean result = Files.deleteIfExists(syslogFile.toPath()); //surround it in try catch block - } catch (IOException e) { + } + catch (IOException e) { throw new RuntimeException(e); } } @@ -150,21 +146,15 @@ public void hdfsWriteTest() { try { startMiniCluster(); - } catch (IOException e) { + } + catch (IOException e) { throw new RuntimeException(e); } Path queueDirectory = new Path(config.getQueueDirectory()); for (int j = 0; j <= 9; j++) { for (int i = 1; i <= 2; i++) { - File syslogFile = new File( - queueDirectory.toUri() - + File.separator - + "testConsumerTopic" - + j - + "." - + i - ); + File syslogFile = new File(queueDirectory.toUri() + File.separator + "testConsumerTopic" + j + "." + i); // generate lastObject from the last record in the file in this test DatumReader userDatumReader = new SpecificDatumReader<>(SyslogRecord.class); @@ -173,26 +163,42 @@ public void hdfsWriteTest() { while (dataFileReader.hasNext()) { lastRecord = dataFileReader.next(lastRecord); } - } catch (IOException e) { + } + catch (IOException e) { throw new RuntimeException(e); } assert lastRecord != null; - RecordOffset lastObject = new RecordOffset("testConsumerTopic", Integer.parseInt(lastRecord.getPartition().toString()), lastRecord.getOffset(), null); // Fetch input parameters from the lastRecord SyslogRecord-object. - LOGGER.debug("\n"+"Last record in the " + syslogFile.getName() + " file:" + "\ntopic: " + lastObject.getTopic() + "\npartition: " + lastObject.getPartition() + "\noffset: " + lastObject.getOffset()); + RecordOffset lastObject = new RecordOffset( + "testConsumerTopic", + Integer.parseInt(lastRecord.getPartition().toString()), + lastRecord.getOffset(), + null + ); // Fetch input parameters from the lastRecord SyslogRecord-object. + LOGGER + .debug( + "\n" + "Last record in the " + syslogFile.getName() + " file:" + "\ntopic: " + + lastObject.getTopic() + "\npartition: " + lastObject.getPartition() + + "\noffset: " + lastObject.getOffset() + ); try (HDFSWrite writer = new HDFSWrite(config, lastObject)) { writer.commit(syslogFile, -1L); // commits the final AVRO-file to HDFS. - } catch (IOException e) { + } + catch (IOException e) { throw new RuntimeException(e); } // Check that the file was stored to HDFS properly. try { Thread.sleep(1000); - hdfsReadCheck("testConsumerTopic", Integer.parseInt(lastRecord.getPartition().toString()), lastRecord.getOffset()); - } catch (IOException e) { + hdfsReadCheck( + "testConsumerTopic", Integer.parseInt(lastRecord.getPartition().toString()), lastRecord.getOffset() + ); + } + catch (IOException e) { throw new RuntimeException(e); - } catch (InterruptedException e) { + } + catch (InterruptedException e) { throw new RuntimeException(e); } @@ -206,8 +212,8 @@ public void hdfsReadCheck(String testConsumerTopic, int partition, long offset) // Check that the files were properly written to HDFS with a read test. String hdfsuri = config.getHdfsuri(); - String path = config.getHdfsPath()+"/"+testConsumerTopic; - String fileName = partition+"."+offset; + String path = config.getHdfsPath() + "/" + testConsumerTopic; + String fileName = partition + "." + offset; // ====== Init HDFS File System Object Configuration conf = new Configuration(); // Set FileSystem URI @@ -222,9 +228,9 @@ public void hdfsReadCheck(String testConsumerTopic, int partition, long offset) FileSystem fs = FileSystem.get(URI.create(hdfsuri), conf); //==== Create directory if not exists - Path workingDir=fs.getWorkingDirectory(); - Path newDirectoryPath= new Path(path); - if(!fs.exists(newDirectoryPath)) { + Path workingDir = fs.getWorkingDirectory(); + Path newDirectoryPath = new Path(path); + if (!fs.exists(newDirectoryPath)) { // Create new Directory fs.mkdirs(newDirectoryPath); // logger.info("Path "+path+" created."); @@ -236,14 +242,19 @@ public void hdfsReadCheck(String testConsumerTopic, int partition, long offset) //Init input stream FSDataInputStream inputStream = fs.open(hdfsreadpath); //The data is in AVRO-format, so it can't be read as a string. - DataFileStream reader = new DataFileStream<>(inputStream, new SpecificDatumReader<>(SyslogRecord.class)); + DataFileStream reader = new DataFileStream<>( + inputStream, + new SpecificDatumReader<>(SyslogRecord.class) + ); SyslogRecord record = null; int looper; if (offset == 8) { looper = 0; - } else if (offset == 13) { + } + else if (offset == 13) { looper = 9; - }else { + } + else { looper = 0; Assertions.fail("The offset of the last record is not 8 or 13, which means a failed test."); } @@ -252,46 +263,129 @@ record = reader.next(record); LOGGER.debug(record.toString()); // Assert records here like it is done in KafkaConsumerTest.avroReader(). if (looper <= 0) { - Assertions.assertEquals("{\"timestamp\": 1650872090804000, \"message\": \"[WARN] 2022-04-25 07:34:50,804 com.teragrep.jla_02.Log4j Log - Log4j warn says hi!\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partition + "\", \"offset\": 0, \"origin\": \"jla-02.default\"}", record.toString()); + Assertions + .assertEquals( + "{\"timestamp\": 1650872090804000, \"message\": \"[WARN] 2022-04-25 07:34:50,804 com.teragrep.jla_02.Log4j Log - Log4j warn says hi!\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partition + "\", \"offset\": 0, \"origin\": \"jla-02.default\"}", + record.toString() + ); looper++; - } else if (looper == 1) { - Assertions.assertEquals("{\"timestamp\": 1650872090806000, \"message\": \"[ERROR] 2022-04-25 07:34:50,806 com.teragrep.jla_02.Log4j Log - Log4j error says hi!\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partition + "\", \"offset\": 1, \"origin\": \"jla-02.default\"}", record.toString()); + } + else if (looper == 1) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872090806000, \"message\": \"[ERROR] 2022-04-25 07:34:50,806 com.teragrep.jla_02.Log4j Log - Log4j error says hi!\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partition + "\", \"offset\": 1, \"origin\": \"jla-02.default\"}", + record.toString() + ); looper++; - } else if (looper == 2) { - Assertions.assertEquals("{\"timestamp\": 1650872090822000, \"message\": \"470647 [Thread-3] INFO com.teragrep.jla_02.Logback Daily - Logback-daily says hi.\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partition + "\", \"offset\": 2, \"origin\": \"jla-02\"}", record.toString()); + } + else if (looper == 2) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872090822000, \"message\": \"470647 [Thread-3] INFO com.teragrep.jla_02.Logback Daily - Logback-daily says hi.\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partition + "\", \"offset\": 2, \"origin\": \"jla-02\"}", + record.toString() + ); looper++; - } else if (looper == 3) { - Assertions.assertEquals("{\"timestamp\": 1650872090822000, \"message\": \"470646 [Thread-3] INFO com.teragrep.jla_02.Logback Audit - Logback-audit says hi.\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partition + "\", \"offset\": 3, \"origin\": \"jla-02\"}", record.toString()); + } + else if (looper == 3) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872090822000, \"message\": \"470646 [Thread-3] INFO com.teragrep.jla_02.Logback Audit - Logback-audit says hi.\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partition + "\", \"offset\": 3, \"origin\": \"jla-02\"}", + record.toString() + ); looper++; - } else if (looper == 4) { - Assertions.assertEquals("{\"timestamp\": 1650872090822000, \"message\": \"470647 [Thread-3] INFO com.teragrep.jla_02.Logback Metric - Logback-metric says hi.\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partition + "\", \"offset\": 4, \"origin\": \"jla-02\"}", record.toString()); + } + else if (looper == 4) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872090822000, \"message\": \"470647 [Thread-3] INFO com.teragrep.jla_02.Logback Metric - Logback-metric says hi.\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partition + "\", \"offset\": 4, \"origin\": \"jla-02\"}", + record.toString() + ); looper++; - } else if (looper == 5) { - Assertions.assertEquals("{\"timestamp\": 1650872092238000, \"message\": \"25.04.2022 07:34:52.238 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info audit says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partition + "\", \"offset\": 5, \"origin\": \"jla-02.default\"}", record.toString()); + } + else if (looper == 5) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872092238000, \"message\": \"25.04.2022 07:34:52.238 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info audit says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partition + "\", \"offset\": 5, \"origin\": \"jla-02.default\"}", + record.toString() + ); looper++; - } else if (looper == 6) { - Assertions.assertEquals("{\"timestamp\": 1650872092239000, \"message\": \"25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info daily says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partition + "\", \"offset\": 6, \"origin\": \"jla-02.default\"}", record.toString()); + } + else if (looper == 6) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872092239000, \"message\": \"25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info daily says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partition + "\", \"offset\": 6, \"origin\": \"jla-02.default\"}", + record.toString() + ); looper++; - } else if (looper == 7) { - Assertions.assertEquals("{\"timestamp\": 1650872092239000, \"message\": \"25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info metric says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partition + "\", \"offset\": 7, \"origin\": \"jla-02.default\"}", record.toString()); + } + else if (looper == 7) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872092239000, \"message\": \"25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info metric says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partition + "\", \"offset\": 7, \"origin\": \"jla-02.default\"}", + record.toString() + ); looper++; - } else if (looper == 8) { - Assertions.assertEquals("{\"timestamp\": 1650872092240000, \"message\": \"25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn audit says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partition + "\", \"offset\": 8, \"origin\": \"jla-02.default\"}", record.toString()); + } + else if (looper == 8) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872092240000, \"message\": \"25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn audit says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partition + "\", \"offset\": 8, \"origin\": \"jla-02.default\"}", + record.toString() + ); looper++; - } else if (looper == 9) { - Assertions.assertEquals("{\"timestamp\": 1650872092240000, \"message\": \"25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn daily says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partition + "\", \"offset\": 9, \"origin\": \"jla-02.default\"}", record.toString()); + } + else if (looper == 9) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872092240000, \"message\": \"25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn daily says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partition + "\", \"offset\": 9, \"origin\": \"jla-02.default\"}", + record.toString() + ); looper++; - } else if (looper == 10) { - Assertions.assertEquals("{\"timestamp\": 1650872092241000, \"message\": \"25.04.2022 07:34:52.241 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn metric says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partition + "\", \"offset\": 10, \"origin\": \"jla-02.default\"}", record.toString()); + } + else if (looper == 10) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872092241000, \"message\": \"25.04.2022 07:34:52.241 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn metric says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partition + "\", \"offset\": 10, \"origin\": \"jla-02.default\"}", + record.toString() + ); looper++; - } else if (looper == 11) { - Assertions.assertEquals("{\"timestamp\": 1650872092241000, \"message\": \"25.04.2022 07:34:52.241 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error audit says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partition + "\", \"offset\": 11, \"origin\": \"jla-02.default\"}", record.toString()); + } + else if (looper == 11) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872092241000, \"message\": \"25.04.2022 07:34:52.241 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error audit says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partition + "\", \"offset\": 11, \"origin\": \"jla-02.default\"}", + record.toString() + ); looper++; - } else if (looper == 12) { - Assertions.assertEquals("{\"timestamp\": 1650872092242000, \"message\": \"25.04.2022 07:34:52.242 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error daily says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partition + "\", \"offset\": 12, \"origin\": \"jla-02.default\"}", record.toString()); + } + else if (looper == 12) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872092242000, \"message\": \"25.04.2022 07:34:52.242 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error daily says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partition + "\", \"offset\": 12, \"origin\": \"jla-02.default\"}", + record.toString() + ); looper++; - } else { - Assertions.assertEquals("{\"timestamp\": 1650872092243000, \"message\": \"25.04.2022 07:34:52.243 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error metric says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partition + "\", \"offset\": 13, \"origin\": \"jla-02.default\"}", record.toString()); + } + else { + Assertions + .assertEquals( + "{\"timestamp\": 1650872092243000, \"message\": \"25.04.2022 07:34:52.243 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error metric says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partition + "\", \"offset\": 13, \"origin\": \"jla-02.default\"}", + record.toString() + ); looper = 0; } } diff --git a/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java b/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java index c2c3af35..371983e5 100644 --- a/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java +++ b/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java @@ -43,25 +43,17 @@ * Teragrep, the applicable Commercial License may apply to this file if you as * a licensee so wish it. */ - package com.teragrep.cfe_39; import com.teragrep.cfe_39.consumers.kafka.KafkaController; import org.apache.avro.file.DataFileReader; import org.apache.avro.io.DatumReader; import org.apache.avro.specific.SpecificDatumReader; -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.consumer.MockConsumer; -import org.apache.kafka.clients.consumer.OffsetResetStrategy; -import org.apache.kafka.common.PartitionInfo; -import org.apache.kafka.common.TopicPartition; import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; import com.teragrep.cfe_39.avro.SyslogRecord; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; @@ -71,6 +63,7 @@ import java.util.*; public class KafkaConsumerTest { + private static final Logger LOGGER = LoggerFactory.getLogger(KafkaConsumerTest.class); // Make sure application.properties has consumer.useMockKafkaConsumer=true enabled for Kafka testing. @@ -83,25 +76,26 @@ public void configTest() { Properties readerKafkaProperties = config.getKafkaConsumerProperties(); // Test extracting useMockKafkaConsumer value from config. - boolean useMockKafkaConsumer = Boolean.parseBoolean( - readerKafkaProperties.getProperty("useMockKafkaConsumer", "false") - ); - LOGGER.debug("useMockKafkaConsumer: "+useMockKafkaConsumer); - } catch (IOException e) { + boolean useMockKafkaConsumer = Boolean + .parseBoolean(readerKafkaProperties.getProperty("useMockKafkaConsumer", "false")); + LOGGER.debug("useMockKafkaConsumer: " + useMockKafkaConsumer); + } + catch (IOException e) { throw new RuntimeException(e); } } - // @Test public void kafkaAndAvroFullTest() throws InterruptedException { Config config = null; try { config = new Config(); - } catch (IOException e){ + } + catch (IOException e) { LOGGER.error("Can't load config: " + e); System.exit(1); - } catch (IllegalArgumentException e) { + } + catch (IllegalArgumentException e) { LOGGER.error("Got invalid config: " + e); System.exit(1); } @@ -111,7 +105,8 @@ public void kafkaAndAvroFullTest() throws InterruptedException { try { int counter = avroReader(1, 2); Assertions.assertEquals(140, counter); - } catch (IOException e) { + } + catch (IOException e) { throw new RuntimeException(e); } cleanup(config, 1, 2); @@ -128,12 +123,7 @@ public int avroReader(int start, int end) throws IOException { for (int j = 0; j <= 9; j++) { for (int i = start; i <= end; i++) { File syslogFile = new File( - queueDirectory.toAbsolutePath() - + File.separator - + "testConsumerTopic" - + j - + "." - + i + queueDirectory.toAbsolutePath() + File.separator + "testConsumerTopic" + j + "." + i ); DatumReader userDatumReader = new SpecificDatumReader<>(SyslogRecord.class); try (DataFileReader dataFileReader = new DataFileReader<>(syslogFile, userDatumReader)) { @@ -145,46 +135,140 @@ public int avroReader(int start, int end) throws IOException { counter++; // All the mock data is generated from a set of 14 records. if (looper <= 0) { - Assertions.assertEquals("{\"timestamp\": 1650872090804000, \"message\": \"[WARN] 2022-04-25 07:34:50,804 com.teragrep.jla_02.Log4j Log - Log4j warn says hi!\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 0, \"origin\": \"jla-02.default\"}", user.toString()); + Assertions + .assertEquals( + "{\"timestamp\": 1650872090804000, \"message\": \"[WARN] 2022-04-25 07:34:50,804 com.teragrep.jla_02.Log4j Log - Log4j warn says hi!\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 0, \"origin\": \"jla-02.default\"}", + user.toString() + ); looper++; - } else if (looper == 1) { - Assertions.assertEquals("{\"timestamp\": 1650872090806000, \"message\": \"[ERROR] 2022-04-25 07:34:50,806 com.teragrep.jla_02.Log4j Log - Log4j error says hi!\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 1, \"origin\": \"jla-02.default\"}", user.toString()); + } + else if (looper == 1) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872090806000, \"message\": \"[ERROR] 2022-04-25 07:34:50,806 com.teragrep.jla_02.Log4j Log - Log4j error says hi!\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 1, \"origin\": \"jla-02.default\"}", + user.toString() + ); looper++; - } else if (looper == 2) { - Assertions.assertEquals("{\"timestamp\": 1650872090822000, \"message\": \"470647 [Thread-3] INFO com.teragrep.jla_02.Logback Daily - Logback-daily says hi.\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 2, \"origin\": \"jla-02\"}", user.toString()); + } + else if (looper == 2) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872090822000, \"message\": \"470647 [Thread-3] INFO com.teragrep.jla_02.Logback Daily - Logback-daily says hi.\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + "\", \"offset\": 2, \"origin\": \"jla-02\"}", + user.toString() + ); looper++; - } else if (looper == 3) { - Assertions.assertEquals("{\"timestamp\": 1650872090822000, \"message\": \"470646 [Thread-3] INFO com.teragrep.jla_02.Logback Audit - Logback-audit says hi.\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 3, \"origin\": \"jla-02\"}", user.toString()); + } + else if (looper == 3) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872090822000, \"message\": \"470646 [Thread-3] INFO com.teragrep.jla_02.Logback Audit - Logback-audit says hi.\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + "\", \"offset\": 3, \"origin\": \"jla-02\"}", + user.toString() + ); looper++; - } else if (looper == 4) { - Assertions.assertEquals("{\"timestamp\": 1650872090822000, \"message\": \"470647 [Thread-3] INFO com.teragrep.jla_02.Logback Metric - Logback-metric says hi.\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 4, \"origin\": \"jla-02\"}", user.toString()); + } + else if (looper == 4) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872090822000, \"message\": \"470647 [Thread-3] INFO com.teragrep.jla_02.Logback Metric - Logback-metric says hi.\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + "\", \"offset\": 4, \"origin\": \"jla-02\"}", + user.toString() + ); looper++; - } else if (looper == 5) { - Assertions.assertEquals("{\"timestamp\": 1650872092238000, \"message\": \"25.04.2022 07:34:52.238 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info audit says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 5, \"origin\": \"jla-02.default\"}", user.toString()); + } + else if (looper == 5) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872092238000, \"message\": \"25.04.2022 07:34:52.238 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info audit says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 5, \"origin\": \"jla-02.default\"}", + user.toString() + ); looper++; - } else if (looper == 6) { - Assertions.assertEquals("{\"timestamp\": 1650872092239000, \"message\": \"25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info daily says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 6, \"origin\": \"jla-02.default\"}", user.toString()); + } + else if (looper == 6) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872092239000, \"message\": \"25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info daily says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 6, \"origin\": \"jla-02.default\"}", + user.toString() + ); looper++; - } else if (looper == 7) { - Assertions.assertEquals("{\"timestamp\": 1650872092239000, \"message\": \"25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info metric says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 7, \"origin\": \"jla-02.default\"}", user.toString()); + } + else if (looper == 7) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872092239000, \"message\": \"25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info metric says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 7, \"origin\": \"jla-02.default\"}", + user.toString() + ); looper++; - } else if (looper == 8) { - Assertions.assertEquals("{\"timestamp\": 1650872092240000, \"message\": \"25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn audit says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 8, \"origin\": \"jla-02.default\"}", user.toString()); + } + else if (looper == 8) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872092240000, \"message\": \"25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn audit says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 8, \"origin\": \"jla-02.default\"}", + user.toString() + ); looper++; - } else if (looper == 9) { - Assertions.assertEquals("{\"timestamp\": 1650872092240000, \"message\": \"25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn daily says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 9, \"origin\": \"jla-02.default\"}", user.toString()); + } + else if (looper == 9) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872092240000, \"message\": \"25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn daily says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 9, \"origin\": \"jla-02.default\"}", + user.toString() + ); looper++; - } else if (looper == 10) { - Assertions.assertEquals("{\"timestamp\": 1650872092241000, \"message\": \"25.04.2022 07:34:52.241 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn metric says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 10, \"origin\": \"jla-02.default\"}", user.toString()); + } + else if (looper == 10) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872092241000, \"message\": \"25.04.2022 07:34:52.241 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn metric says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 10, \"origin\": \"jla-02.default\"}", + user.toString() + ); looper++; - } else if (looper == 11) { - Assertions.assertEquals("{\"timestamp\": 1650872092241000, \"message\": \"25.04.2022 07:34:52.241 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error audit says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 11, \"origin\": \"jla-02.default\"}", user.toString()); + } + else if (looper == 11) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872092241000, \"message\": \"25.04.2022 07:34:52.241 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error audit says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 11, \"origin\": \"jla-02.default\"}", + user.toString() + ); looper++; - } else if (looper == 12) { - Assertions.assertEquals("{\"timestamp\": 1650872092242000, \"message\": \"25.04.2022 07:34:52.242 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error daily says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 12, \"origin\": \"jla-02.default\"}", user.toString()); + } + else if (looper == 12) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872092242000, \"message\": \"25.04.2022 07:34:52.242 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error daily says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 12, \"origin\": \"jla-02.default\"}", + user.toString() + ); looper++; - } else { - Assertions.assertEquals("{\"timestamp\": 1650872092243000, \"message\": \"25.04.2022 07:34:52.243 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error metric says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + partitionCounter + "\", \"offset\": 13, \"origin\": \"jla-02.default\"}", user.toString()); + } + else { + Assertions + .assertEquals( + "{\"timestamp\": 1650872092243000, \"message\": \"25.04.2022 07:34:52.243 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error metric says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 13, \"origin\": \"jla-02.default\"}", + user.toString() + ); looper = 0; partitionCounter++; } @@ -201,7 +285,8 @@ public void debugger() { try { int counter = avroReader(1, 2); Assertions.assertEquals(140, counter); - } catch (IOException e) { + } + catch (IOException e) { throw new RuntimeException(e); } } @@ -212,16 +297,12 @@ public void cleanup(Config config, int start, int end) { for (int j = 0; j <= 9; j++) { for (int i = start; i <= end; i++) { File syslogFile = new File( - queueDirectory.toAbsolutePath() - + File.separator - + "testConsumerTopic" - + j - + "." - + i + queueDirectory.toAbsolutePath() + File.separator + "testConsumerTopic" + j + "." + i ); try { boolean result = Files.deleteIfExists(syslogFile.toPath()); //surround it in try catch block - } catch (IOException e) { + } + catch (IOException e) { throw new RuntimeException(e); } } From b12bb1423c085c18f5152a9459b0caec25db8457 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Wed, 29 May 2024 15:33:10 +0300 Subject: [PATCH 074/146] Implemented logger call guards. --- .../consumers/kafka/DatabaseOutput.java | 81 +++++++++++-------- .../cfe_39/consumers/kafka/KafkaReader.java | 4 +- .../com/teragrep/cfe_39/CombinedFullTest.java | 8 +- .../java/com/teragrep/cfe_39/HdfsTest.java | 18 +++-- .../teragrep/cfe_39/KafkaConsumerTest.java | 6 +- 5 files changed, 71 insertions(+), 46 deletions(-) diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java index bb714c1d..792d47b8 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java @@ -165,11 +165,14 @@ public void accept(List recordOffsetObjectList) { long thisTime = Instant.now().toEpochMilli(); long ftook = thisTime - lastTimeCalled; topicCounter.setKafkaLatency(ftook); - LOGGER - .debug( - "Fuura searching your batch for <[{}]> with records <{}> and took <{}> milliseconds. <{}> EPS. ", - table, recordOffsetObjectList.size(), (ftook), (recordOffsetObjectList.size() * 1000L / ftook) - ); + if (LOGGER.isDebugEnabled()) { + LOGGER + .debug( + "Fuura searching your batch for <[{}]> with records <{}> and took <{}> milliseconds. <{}> EPS. ", + table, recordOffsetObjectList.size(), (ftook), + (recordOffsetObjectList.size() * 1000L / ftook) + ); + } long batchBytes = 0L; /* The recordOffsetObjectList loop will go through all the objects in the list. @@ -205,20 +208,24 @@ public void accept(List recordOffsetObjectList) { syslogAvroWriter.getFileSize(), (RecordOffset) lastObject ); if (fileCommitted) { - LOGGER - .debug( - "Target file size reached, file <{}> stored to <{}> in HDFS", - syslogFile.getName(), - lastObject.getTopic() + "/" + lastObject.getPartition() + "." - + lastObject.getOffset() - ); + if (LOGGER.isDebugEnabled()) { + LOGGER + .debug( + "Target file size reached, file <{}> stored to <{}> in HDFS", + syslogFile.getName(), + lastObject.getTopic() + "/" + lastObject.getPartition() + "." + + lastObject.getOffset() + ); + } } else { - LOGGER - .debug( - "Target file size not yet reached, continuing writing records to <{}>.", - syslogFile.getName() - ); + if (LOGGER.isDebugEnabled()) { + LOGGER + .debug( + "Target file size not yet reached, continuing writing records to <{}>.", + syslogFile.getName() + ); + } } } else { @@ -284,20 +291,24 @@ public void accept(List recordOffsetObjectList) { syslogAvroWriter.getFileSize() + capacity, (RecordOffset) lastObject ); if (fileCommitted) { - LOGGER - .debug( - "Target file size reached, file <{}> stored to <{}> in HDFS", - syslogFile.getName(), - lastObject.getTopic() + "/" + lastObject.getPartition() + "." - + lastObject.getOffset() - ); + if (LOGGER.isDebugEnabled()) { + LOGGER + .debug( + "Target file size reached, file <{}> stored to <{}> in HDFS", + syslogFile.getName(), + lastObject.getTopic() + "/" + lastObject.getPartition() + "." + + lastObject.getOffset() + ); + } } else { - LOGGER - .debug( - "Target file size not yet reached, continuing writing records to <{}>.", - syslogFile.getName() - ); + if (LOGGER.isDebugEnabled()) { + LOGGER + .debug( + "Target file size not yet reached, continuing writing records to <{}>.", + syslogFile.getName() + ); + } } // if more records can be inserted, update epochMicros_last with the timestamp of the last inserted record. epochMicros_last = epochMicros; @@ -345,11 +356,13 @@ public void accept(List recordOffsetObjectList) { topicCounter.addToTotalBytes(batchBytes); topicCounter.addToTotalRecords(recordOffsetObjectList.size()); - LOGGER - .debug( - "Sent batch for <[{}]> with records <{}> and size <{}> KB took <{}> milliseconds. <{}> RPS. <{}> KB/s ", - table, recordOffsetObjectList.size(), batchBytes / 1024, (took), rps, bps / 1024 - ); + if (LOGGER.isDebugEnabled()) { + LOGGER + .debug( + "Sent batch for <[{}]> with records <{}> and size <{}> KB took <{}> milliseconds. <{}> RPS. <{}> KB/s ", + table, recordOffsetObjectList.size(), batchBytes / 1024, (took), rps, bps / 1024 + ); + } lastTimeCalled = Instant.now().toEpochMilli(); } diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReader.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReader.java index 17febbe4..a60d9d25 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReader.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaReader.java @@ -82,7 +82,9 @@ public void read() { List recordOffsetObjectList = new ArrayList<>(); while (kafkaRecordsIterator.hasNext()) { ConsumerRecord record = kafkaRecordsIterator.next(); - LOGGER.debug("adding from offset: <{}>", record.offset()); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug("adding from offset: <{}>", record.offset()); + } recordOffsetObjectList .add(new RecordOffset(record.topic(), record.partition(), record.offset(), record.value())); } diff --git a/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java b/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java index 704211cf..148044c5 100644 --- a/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java +++ b/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java @@ -251,7 +251,9 @@ public void hdfsReadCheck() throws IOException { LOGGER.info("\nReading records from file {}:", hdfsreadpath.toString()); while (reader.hasNext()) { record = reader.next(record); - LOGGER.debug(record.toString()); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug(record.toString()); + } // Assert records here like it is done in KafkaConsumerTest.avroReader(). if (looper <= 0) { Assertions @@ -448,7 +450,9 @@ private void insertMockFiles() throws IOException { // Add conditions if file filtering is required for tests. fs.copyFromLocalFile(readPath, hdfswritepath); LOGGER.debug("End Write file into hdfs"); - LOGGER.debug("\nFile committed to HDFS, file writepath should be: {}\n", hdfswritepath.toString()); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug("\nFile committed to HDFS, file writepath should be: {}\n", hdfswritepath.toString()); + } } fs.close(); } diff --git a/src/test/java/com/teragrep/cfe_39/HdfsTest.java b/src/test/java/com/teragrep/cfe_39/HdfsTest.java index 1613025f..5ac91c9a 100644 --- a/src/test/java/com/teragrep/cfe_39/HdfsTest.java +++ b/src/test/java/com/teragrep/cfe_39/HdfsTest.java @@ -175,12 +175,14 @@ public void hdfsWriteTest() { lastRecord.getOffset(), null ); // Fetch input parameters from the lastRecord SyslogRecord-object. - LOGGER - .debug( - "\n" + "Last record in the " + syslogFile.getName() + " file:" + "\ntopic: " - + lastObject.getTopic() + "\npartition: " + lastObject.getPartition() - + "\noffset: " + lastObject.getOffset() - ); + if (LOGGER.isDebugEnabled()) { + LOGGER + .debug( + "\n" + "Last record in the " + syslogFile.getName() + " file:" + "\ntopic: " + + lastObject.getTopic() + "\npartition: " + lastObject.getPartition() + + "\noffset: " + lastObject.getOffset() + ); + } try (HDFSWrite writer = new HDFSWrite(config, lastObject)) { writer.commit(syslogFile, -1L); // commits the final AVRO-file to HDFS. } @@ -260,7 +262,9 @@ else if (offset == 13) { } while (reader.hasNext()) { record = reader.next(record); - LOGGER.debug(record.toString()); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug(record.toString()); + } // Assert records here like it is done in KafkaConsumerTest.avroReader(). if (looper <= 0) { Assertions diff --git a/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java b/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java index 371983e5..6fb73e06 100644 --- a/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java +++ b/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java @@ -130,8 +130,10 @@ public int avroReader(int start, int end) throws IOException { SyslogRecord user = null; while (dataFileReader.hasNext()) { user = dataFileReader.next(user); - LOGGER.debug(syslogFile.getPath()); - LOGGER.debug(user.toString()); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug(syslogFile.getPath()); + LOGGER.debug(user.toString()); + } counter++; // All the mock data is generated from a set of 14 records. if (looper <= 0) { From 57c6805d036db4ea9c7fca8aa81a5bfbbdd5d2b6 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Wed, 29 May 2024 16:01:00 +0300 Subject: [PATCH 075/146] Renamed MockKafkaConsumerFactoryTemp to MockKafkaConsumerFactory --- .../teragrep/cfe_39/consumers/kafka/KafkaController.java | 2 +- ...nsumerFactoryTemp.java => MockKafkaConsumerFactory.java} | 6 +++--- .../teragrep/cfe_39/consumers/kafka/ReadCoordinator.java | 4 ++-- 3 files changed, 6 insertions(+), 6 deletions(-) rename src/main/java/com/teragrep/cfe_39/consumers/kafka/{MockKafkaConsumerFactoryTemp.java => MockKafkaConsumerFactory.java} (99%) diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java index 7b208850..20ad28dd 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java @@ -85,7 +85,7 @@ public KafkaController(Config config) { this.useMockKafkaConsumer = Boolean .parseBoolean(readerKafkaProperties.getProperty("useMockKafkaConsumer", "false")); if (useMockKafkaConsumer) { - this.kafkaConsumer = MockKafkaConsumerFactoryTemp.getConsumer(0); // A consumer used only for scanning the available topics to be allocated to consumers running in different threads (thus 0 as input parameter). + this.kafkaConsumer = MockKafkaConsumerFactory.getConsumer(0); // A consumer used only for scanning the available topics to be allocated to consumers running in different threads (thus 0 as input parameter). } else { this.kafkaConsumer = new KafkaConsumer<>( diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactory.java similarity index 99% rename from src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java rename to src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactory.java index f7a9d81a..564ba4b7 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactoryTemp.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactory.java @@ -63,11 +63,11 @@ * @since 08/06/2022 * @author Mikko Kortelainen */ -public class MockKafkaConsumerFactoryTemp { +public class MockKafkaConsumerFactory { - final static private Logger LOGGER = LoggerFactory.getLogger(MockKafkaConsumerFactoryTemp.class); + final static private Logger LOGGER = LoggerFactory.getLogger(MockKafkaConsumerFactory.class); - private MockKafkaConsumerFactoryTemp() { + private MockKafkaConsumerFactory() { } private static void generateEvents(MockConsumer consumer, String topicName, int partition) { diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinator.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinator.java index fb9f8d8e..f962ecf9 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinator.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinator.java @@ -87,10 +87,10 @@ private KafkaReader createKafkaReader( if (useMockKafkaConsumer) { // Mock kafka consumer is enabled, create mock consumers with assigned partitions that are not overlapping with each other. String name = Thread.currentThread().getName(); // Use thread name to identify which thread is running the code. if (Objects.equals(name, "testConsumerTopic1")) { - kafkaConsumer = MockKafkaConsumerFactoryTemp.getConsumer(1); // creates a Kafka MockConsumer that has the odd numbered partitions assigned to it. + kafkaConsumer = MockKafkaConsumerFactory.getConsumer(1); // creates a Kafka MockConsumer that has the odd numbered partitions assigned to it. } else { - kafkaConsumer = MockKafkaConsumerFactoryTemp.getConsumer(2); // creates a Kafka MockConsumer that has the even numbered partitions assigned to it. + kafkaConsumer = MockKafkaConsumerFactory.getConsumer(2); // creates a Kafka MockConsumer that has the even numbered partitions assigned to it. } } else { // Mock kafka consumer is disabled, subscribe method should handle assigning the partitions automatically to the consumer based on group id parameters of readerKafkaProperties. From 75362246108b420bf91da955889376aaaf157162 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Thu, 30 May 2024 11:32:27 +0300 Subject: [PATCH 076/146] Added @VisibleForTesting to MockKafkaConsumerFactory --- .../cfe_39/consumers/kafka/MockKafkaConsumerFactory.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactory.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactory.java index 564ba4b7..a5223b17 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactory.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactory.java @@ -45,6 +45,7 @@ */ package com.teragrep.cfe_39.consumers.kafka; +import com.google.common.annotations.VisibleForTesting; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.MockConsumer; @@ -63,6 +64,7 @@ * @since 08/06/2022 * @author Mikko Kortelainen */ +@VisibleForTesting public class MockKafkaConsumerFactory { final static private Logger LOGGER = LoggerFactory.getLogger(MockKafkaConsumerFactory.class); From 0d536bee72cbc9b97b70dd731f794381fadc3e1f Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Thu, 30 May 2024 14:08:34 +0300 Subject: [PATCH 077/146] Added missing FIXME tag --- .../com/teragrep/cfe_39/consumers/kafka/ReadCoordinator.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinator.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinator.java index f962ecf9..c5c45694 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinator.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinator.java @@ -132,7 +132,7 @@ public void run() { } } - // remove? + // FIXME: remove? public void stop() { run = false; } From 75cb6183b3d3a3e65c47733727ac2a6d92c3b089 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Thu, 30 May 2024 14:41:14 +0300 Subject: [PATCH 078/146] Changed to using UncheckedIOException if necessary. --- .../com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java index 792d47b8..88a72da3 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java @@ -318,7 +318,7 @@ public void accept(List recordOffsetObjectList) { } } catch (IOException e) { - throw new RuntimeException(e); + throw new UncheckedIOException(e); } } @@ -332,7 +332,7 @@ public void accept(List recordOffsetObjectList) { } } catch (IOException e) { - throw new RuntimeException(e); + throw new UncheckedIOException(e); } // Measures performance of code that is between start and end. From e2014700860ace623dc8f89283ed47f05bd2d924 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Fri, 31 May 2024 15:32:02 +0300 Subject: [PATCH 079/146] Renamed KafkaController.java to HdfsDataIngestion.java --- .../kafka/{KafkaController.java => HdfsDataIngestion.java} | 7 ++++--- src/test/java/com/teragrep/cfe_39/CombinedFullTest.java | 6 +++--- src/test/java/com/teragrep/cfe_39/HdfsTest.java | 6 +++--- src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java | 6 +++--- 4 files changed, 13 insertions(+), 12 deletions(-) rename src/main/java/com/teragrep/cfe_39/consumers/kafka/{KafkaController.java => HdfsDataIngestion.java} (96%) diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HdfsDataIngestion.java similarity index 96% rename from src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java rename to src/main/java/com/teragrep/cfe_39/consumers/kafka/HdfsDataIngestion.java index 20ad28dd..4ceed3ee 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/KafkaController.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HdfsDataIngestion.java @@ -64,9 +64,10 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; -public class KafkaController { +// Ingests data for HDFS database, periodically scans kafka for new topics based on config.getQueueTopicPattern() and creates kafka topic consumer groups for the new topics that will store the records to HDFS. +public class HdfsDataIngestion { - private static final Logger LOGGER = LoggerFactory.getLogger(KafkaController.class); + private static final Logger LOGGER = LoggerFactory.getLogger(HdfsDataIngestion.class); private final Config config; private final org.apache.kafka.clients.consumer.Consumer kafkaConsumer; private final List threads = new ArrayList<>(); @@ -77,7 +78,7 @@ public class KafkaController { private final int numOfConsumers; private Map hdfsStartOffsets; - public KafkaController(Config config) { + public HdfsDataIngestion(Config config) { keepRunning = true; this.config = config; Properties readerKafkaProperties = config.getKafkaConsumerProperties(); diff --git a/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java b/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java index 148044c5..108f174b 100644 --- a/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java +++ b/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java @@ -46,7 +46,7 @@ package com.teragrep.cfe_39; import com.teragrep.cfe_39.avro.SyslogRecord; -import com.teragrep.cfe_39.consumers.kafka.KafkaController; +import com.teragrep.cfe_39.consumers.kafka.HdfsDataIngestion; import org.apache.avro.file.DataFileStream; import org.apache.avro.specific.SpecificDatumReader; import org.apache.hadoop.conf.Configuration; @@ -113,9 +113,9 @@ public static void teardownMiniCluster() { public void kafkaAndAvroFullTest() throws InterruptedException, IOException { insertMockFiles(); // Maybe add modifier that allows adding or not adding the mock file. config.setMaximumFileSize(3000); // 10 loops (140 records) are in use at the moment, and that is sized at 36,102 bytes. - KafkaController kafkaController = new KafkaController(config); + HdfsDataIngestion hdfsDataIngestion = new HdfsDataIngestion(config); Thread.sleep(10000); - kafkaController.run(); + hdfsDataIngestion.run(); /* The avro files should be committed to HDFS now. Check the committed files for any errors. There should be 20 files, 10 partitions with each having 2 files assigned to them. hdfsReadCheck(); does not work properly if pruning is enabled and prune offset is set too low, which causes the records to be pruned from the database.*/ diff --git a/src/test/java/com/teragrep/cfe_39/HdfsTest.java b/src/test/java/com/teragrep/cfe_39/HdfsTest.java index 5ac91c9a..6005caa6 100644 --- a/src/test/java/com/teragrep/cfe_39/HdfsTest.java +++ b/src/test/java/com/teragrep/cfe_39/HdfsTest.java @@ -47,7 +47,7 @@ import com.teragrep.cfe_39.avro.SyslogRecord; import com.teragrep.cfe_39.consumers.kafka.HDFSWrite; -import com.teragrep.cfe_39.consumers.kafka.KafkaController; +import com.teragrep.cfe_39.consumers.kafka.HdfsDataIngestion; import com.teragrep.cfe_39.consumers.kafka.RecordOffset; import org.apache.avro.file.DataFileReader; import org.apache.avro.file.DataFileStream; @@ -94,8 +94,8 @@ public static void generateTestData() throws IOException, InterruptedException { } startMiniCluster(); config.setMaximumFileSize(3000); // 10 loops (140 records) are in use at the moment, and that is sized at 36,102 bytes. - KafkaController kafkaController = new KafkaController(config); - kafkaController.run(); + HdfsDataIngestion hdfsDataIngestion = new HdfsDataIngestion(config); + hdfsDataIngestion.run(); } public static void startMiniCluster() throws IOException { diff --git a/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java b/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java index 6fb73e06..85432db8 100644 --- a/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java +++ b/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java @@ -45,7 +45,7 @@ */ package com.teragrep.cfe_39; -import com.teragrep.cfe_39.consumers.kafka.KafkaController; +import com.teragrep.cfe_39.consumers.kafka.HdfsDataIngestion; import org.apache.avro.file.DataFileReader; import org.apache.avro.io.DatumReader; import org.apache.avro.specific.SpecificDatumReader; @@ -100,8 +100,8 @@ public void kafkaAndAvroFullTest() throws InterruptedException { System.exit(1); } config.setMaximumFileSize(3000); // 10 loops (140 records) are in use at the moment, and that is sized at 36,102 bytes. - KafkaController kafkaController = new KafkaController(config); - kafkaController.run(); + HdfsDataIngestion hdfsDataIngestion = new HdfsDataIngestion(config); + hdfsDataIngestion.run(); try { int counter = avroReader(1, 2); Assertions.assertEquals(140, counter); From 70c6f26b298d4bfd93b2e8e0cdf47f5f4c5ce7c3 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Fri, 31 May 2024 16:11:08 +0300 Subject: [PATCH 080/146] Refactored Offset abstract class into an interface from abstract class. Changed RecordOffset and NullOffset to implement Offset. --- .../consumers/kafka/DatabaseOutput.java | 25 +++++++++---------- .../cfe_39/consumers/kafka/HDFSWrite.java | 8 +++--- .../cfe_39/consumers/kafka/NullOffset.java | 11 ++++---- .../cfe_39/consumers/kafka/Offset.java | 17 +++++-------- .../cfe_39/consumers/kafka/RecordOffset.java | 15 +++++++---- .../java/com/teragrep/cfe_39/HdfsTest.java | 4 +-- 6 files changed, 40 insertions(+), 40 deletions(-) diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java index 88a72da3..38d180e5 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java @@ -133,7 +133,7 @@ boolean committedToHdfs(long fileSize, RecordOffset recordOffsetObject) { } // This part defines a new empty file to which the new AVRO-serialized records are stored until it again hits the 64M size limit. - writableQueue.setQueueNamePrefix(recordOffsetObject.getTopic() + recordOffsetObject.getPartition()); + writableQueue.setQueueNamePrefix(recordOffsetObject.topic() + recordOffsetObject.partition()); syslogFile = writableQueue.getNextWritableFile(); syslogAvroWriter = new SyslogAvroWriter(syslogFile); return true; @@ -187,7 +187,7 @@ public void accept(List recordOffsetObjectList) { // Initializing syslogAvroWriter and lastObject. if (syslogAvroWriter == null && lastObject.isNull()) { try { - writableQueue.setQueueNamePrefix(recordOffsetObject.getTopic() + recordOffsetObject.getPartition()); + writableQueue.setQueueNamePrefix(recordOffsetObject.topic() + recordOffsetObject.partition()); syslogFile = writableQueue.getNextWritableFile(); // The HDFS filename is only finalized when the AVRO-serialized file is finalized, because every Kafka-record added to the file is going to change the offset that is going to be used for the filename. syslogAvroWriter = new SyslogAvroWriter(syslogFile); @@ -200,8 +200,8 @@ public void accept(List recordOffsetObjectList) { else { try { if ( - lastObject.getTopic().equals(recordOffsetObject.getTopic()) - & lastObject.getPartition().equals(recordOffsetObject.getPartition()) + lastObject.topic().equals(recordOffsetObject.topic()) + & lastObject.partition().equals(recordOffsetObject.partition()) ) { // Records left to consume in the current partition. boolean fileCommitted = committedToHdfs( @@ -213,8 +213,8 @@ public void accept(List recordOffsetObjectList) { .debug( "Target file size reached, file <{}> stored to <{}> in HDFS", syslogFile.getName(), - lastObject.getTopic() + "/" + lastObject.getPartition() + "." - + lastObject.getOffset() + lastObject.topic() + "/" + lastObject.partition() + "." + + lastObject.offset() ); } } @@ -235,8 +235,7 @@ public void accept(List recordOffsetObjectList) { writer.commit(syslogFile, epochMicros_last); // This part defines a new empty file to which the new AVRO-serialized records are stored until it again hits the 64M size limit. - writableQueue - .setQueueNamePrefix(recordOffsetObject.getTopic() + recordOffsetObject.getPartition()); + writableQueue.setQueueNamePrefix(recordOffsetObject.topic() + recordOffsetObject.partition()); syslogFile = writableQueue.getNextWritableFile(); syslogAvroWriter = new SyslogAvroWriter(syslogFile); } @@ -246,7 +245,7 @@ public void accept(List recordOffsetObjectList) { } } - byte[] byteArray = recordOffsetObject.getRecord(); // loads the byte[] contained in recordOffsetObject.getRecord() to byteArray. + byte[] byteArray = recordOffsetObject.record(); // loads the byte[] contained in recordOffsetObject.getRecord() to byteArray. batchBytes = batchBytes + byteArray.length; InputStream inputStream = new ByteArrayInputStream(byteArray); rfc5424Frame.load(inputStream); @@ -275,8 +274,8 @@ public void accept(List recordOffsetObjectList) { .setStream(rfc5424Frame.structuredData.getValue(teragrepStreamName).toString()) // Or is sourcetype/stream supposed to be rfc5424Frame.appName.toString() instead? .setHost(rfc5424Frame.hostname.toString()) .setInput(new String(source, StandardCharsets.UTF_8)) - .setPartition(recordOffsetObject.getPartition().toString()) - .setOffset(recordOffsetObject.getOffset()) + .setPartition(recordOffsetObject.partition().toString()) + .setOffset(recordOffsetObject.offset()) .setOrigin(new String(origin, StandardCharsets.UTF_8)) .build(); @@ -296,8 +295,8 @@ public void accept(List recordOffsetObjectList) { .debug( "Target file size reached, file <{}> stored to <{}> in HDFS", syslogFile.getName(), - lastObject.getTopic() + "/" + lastObject.getPartition() + "." - + lastObject.getOffset() + lastObject.topic() + "/" + lastObject.partition() + "." + + lastObject.offset() ); } } diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java index 2df8b2b4..ac3d9c45 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java @@ -81,8 +81,8 @@ public HDFSWrite(Config config, RecordOffset lastObject) throws IOException { /* The filepath should be something like hdfs:///opt/teragrep/cfe_39/srv/topic_name/0.12345 where 12345 is offset and 0 the partition. In other words the directory named topic_name holds files that are named and arranged based on partition and the partition's offset. Every partition has its own set of unique offset values. These values should be fetched from config and other input parameters (topic+partition+offset).*/ - path = config.getHdfsPath() + "/" + lastObject.topic; - fileName = lastObject.partition + "." + lastObject.offset; // filename should be constructed from partition and offset. + path = config.getHdfsPath() + "/" + lastObject.topic(); + fileName = lastObject.partition() + "." + lastObject.offset(); // filename should be constructed from partition and offset. // ====== Init HDFS File System Object conf = new Configuration(); @@ -107,8 +107,8 @@ These values should be fetched from config and other input parameters (topic+par // Code for initializing the class for kerberized HDFS database usage. hdfsuri = config.getHdfsuri(); - path = config.getHdfsPath() + "/" + lastObject.topic; - fileName = lastObject.partition + "." + lastObject.offset; + path = config.getHdfsPath() + "/" + lastObject.topic(); + fileName = lastObject.partition() + "." + lastObject.offset(); // set kerberos host and realm System.setProperty("java.security.krb5.realm", config.getKerberosRealm()); diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/NullOffset.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/NullOffset.java index 235a7ff4..dfb304e5 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/NullOffset.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/NullOffset.java @@ -45,7 +45,8 @@ */ package com.teragrep.cfe_39.consumers.kafka; -public class NullOffset extends Offset { +// Null object design pattern, used to create null offset objects. +public final class NullOffset implements Offset { @Override public boolean isNull() { @@ -53,22 +54,22 @@ public boolean isNull() { } @Override - public String getTopic() { + public String topic() { return "Not available"; } @Override - public Integer getPartition() { + public Integer partition() { return 0; } @Override - public Long getOffset() { + public Long offset() { return 0L; } @Override - public byte[] getRecord() { + public byte[] record() { return new byte[0]; } } diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/Offset.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/Offset.java index 8dd194e5..1d808ba1 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/Offset.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/Offset.java @@ -45,20 +45,15 @@ */ package com.teragrep.cfe_39.consumers.kafka; -public abstract class Offset { +public interface Offset { - protected String topic; - protected Integer partition; - protected Long offset; - protected byte[] record; + public boolean isNull(); - public abstract boolean isNull(); + public String topic(); - public abstract String getTopic(); + public Integer partition(); - public abstract Integer getPartition(); + public Long offset(); - public abstract Long getOffset(); - - public abstract byte[] getRecord(); + public byte[] record(); } diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/RecordOffset.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/RecordOffset.java index 72332842..e2136292 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/RecordOffset.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/RecordOffset.java @@ -46,7 +46,12 @@ package com.teragrep.cfe_39.consumers.kafka; // This is the class for handling the Kafka record topic/partition/offset data that are required for HDFS storage. -public class RecordOffset extends Offset { +public final class RecordOffset implements Offset { + + private final String topic; + private final int partition; + private final long offset; + private final byte[] record; public RecordOffset(String topic, int partition, long offset, byte[] record) { this.topic = topic; @@ -61,22 +66,22 @@ public boolean isNull() { } @Override - public String getTopic() { + public String topic() { return topic; } @Override - public Integer getPartition() { + public Integer partition() { return partition; } @Override - public Long getOffset() { + public Long offset() { return offset; } @Override - public byte[] getRecord() { + public byte[] record() { return record; } } diff --git a/src/test/java/com/teragrep/cfe_39/HdfsTest.java b/src/test/java/com/teragrep/cfe_39/HdfsTest.java index 6005caa6..49060700 100644 --- a/src/test/java/com/teragrep/cfe_39/HdfsTest.java +++ b/src/test/java/com/teragrep/cfe_39/HdfsTest.java @@ -179,8 +179,8 @@ public void hdfsWriteTest() { LOGGER .debug( "\n" + "Last record in the " + syslogFile.getName() + " file:" + "\ntopic: " - + lastObject.getTopic() + "\npartition: " + lastObject.getPartition() - + "\noffset: " + lastObject.getOffset() + + lastObject.topic() + "\npartition: " + lastObject.partition() + + "\noffset: " + lastObject.offset() ); } try (HDFSWrite writer = new HDFSWrite(config, lastObject)) { From 2f64e8b8e30e1221aeb01e5479c11fa096d53ccb Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Mon, 3 Jun 2024 09:35:12 +0300 Subject: [PATCH 081/146] Moved DurationStatistics usage from class field to a local variable. --- .../cfe_39/consumers/kafka/HdfsDataIngestion.java | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HdfsDataIngestion.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HdfsDataIngestion.java index 4ceed3ee..e2c1d2fe 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HdfsDataIngestion.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HdfsDataIngestion.java @@ -72,7 +72,6 @@ public class HdfsDataIngestion { private final org.apache.kafka.clients.consumer.Consumer kafkaConsumer; private final List threads = new ArrayList<>(); private final Set activeTopics = new HashSet<>(); - private final DurationStatistics durationStatistics = new DurationStatistics(); private boolean keepRunning; private boolean useMockKafkaConsumer; private final int numOfConsumers; @@ -100,7 +99,8 @@ public HdfsDataIngestion(Config config) { public void run() throws InterruptedException { - // register duration statistics + // Initialize and register duration statistics + DurationStatistics durationStatistics = new DurationStatistics(); durationStatistics.register(); // register per topic counting @@ -145,8 +145,12 @@ public void run() throws InterruptedException { } // Creates kafka topic consumer based on input parameters. - private void createReader(String topic, List listPartitionInfo, List topicCounters) - throws SQLException { + private void createReader( + String topic, + List listPartitionInfo, + List topicCounters, + DurationStatistics durationStatistics + ) throws SQLException { // Create a new topicCounter object for the topic that has not been added to topicCounters-list yet. TopicCounter topicCounter = new TopicCounter(topic); @@ -203,7 +207,7 @@ private void topicScan(DurationStatistics durationStatistics, List foundPartitions.forEach((k, v) -> { LOGGER.debug("Activating topic <{}>", k); try { - createReader(k, v, topicCounters); + createReader(k, v, topicCounters, durationStatistics); activeTopics.add(k); durationStatistics.addAndGetThreads(1); } From d7cab0cb854863185473e5854b89aeeb3b0bc55c Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Mon, 3 Jun 2024 09:38:04 +0300 Subject: [PATCH 082/146] Removed redundant public modifiers. --- .../com/teragrep/cfe_39/consumers/kafka/Offset.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/Offset.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/Offset.java index 1d808ba1..17b2fdb9 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/Offset.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/Offset.java @@ -47,13 +47,13 @@ public interface Offset { - public boolean isNull(); + boolean isNull(); - public String topic(); + String topic(); - public Integer partition(); + Integer partition(); - public Long offset(); + Long offset(); - public byte[] record(); + byte[] record(); } From c1c455cba47794cdf3379116ef2932bf61a22128 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Mon, 3 Jun 2024 15:05:53 +0300 Subject: [PATCH 083/146] Refactored Offset, RecordOffset and NullOffset further by implementing printer for topic, partition and offset values. --- .../consumers/kafka/DatabaseOutput.java | 54 +++++++++++-------- .../cfe_39/consumers/kafka/HDFSWrite.java | 11 ++-- .../cfe_39/consumers/kafka/NullOffset.java | 18 ++----- .../cfe_39/consumers/kafka/Offset.java | 8 +-- .../cfe_39/consumers/kafka/RecordOffset.java | 22 +++----- .../java/com/teragrep/cfe_39/HdfsTest.java | 8 +-- 6 files changed, 56 insertions(+), 65 deletions(-) diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java index 38d180e5..6a22875f 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java @@ -45,6 +45,7 @@ */ package com.teragrep.cfe_39.consumers.kafka; +import com.google.gson.*; import com.teragrep.cfe_39.Config; import com.teragrep.cfe_39.avro.SyslogRecord; import com.teragrep.cfe_39.consumers.kafka.queue.WritableQueue; @@ -120,7 +121,7 @@ public class DatabaseOutput implements Consumer> { } // Checks that the filesize stays under the defined maximum file size. If the file is about to go over target limit commits the file to HDFS and returns true, otherwise does nothing and returns false. - boolean committedToHdfs(long fileSize, RecordOffset recordOffsetObject) { + boolean committedToHdfs(long fileSize, RecordOffset recordOffsetObject, JsonObject recordOffsetObjectJo) { try { // If the syslogAvroWriter is already initialized, check the filesize so it doesn't go above maximumFileSize. if (fileSize > maximumFileSize) { @@ -128,12 +129,13 @@ boolean committedToHdfs(long fileSize, RecordOffset recordOffsetObject) { // This part closes the writing of now "complete" AVRO-file and stores the file to HDFS. syslogAvroWriter.close(); - try (HDFSWrite writer = new HDFSWrite(config, recordOffsetObject)) { + try (HDFSWrite writer = new HDFSWrite(config, recordOffsetObject, recordOffsetObjectJo)) { writer.commit(syslogFile, epochMicros_last); // commits the final AVRO-file to HDFS. } // This part defines a new empty file to which the new AVRO-serialized records are stored until it again hits the 64M size limit. - writableQueue.setQueueNamePrefix(recordOffsetObject.topic() + recordOffsetObject.partition()); + writableQueue + .setQueueNamePrefix(recordOffsetObjectJo.get("topic").getAsString() + recordOffsetObjectJo.get("partition").getAsString()); syslogFile = writableQueue.getNextWritableFile(); syslogAvroWriter = new SyslogAvroWriter(syslogFile); return true; @@ -180,18 +182,24 @@ public void accept(List recordOffsetObjectList) { When the file size is about to go above 64M, commit the file into HDFS using the latest topic/partition/offset values as the filename and start fresh with a new empty AVRO-file. Serialize the object that was going to make the file go above 64M into the now empty AVRO-file and continue the loop. TODO: If the prod-environment recordOffsetObjectList ordering is different from what it is in the test environment, add a function that reorders the list based on partition and offset (or better yet, make several AVRO-files that are being used at the same time rather than doing it one AVRO-file at a time as the offset ordering within partitions should always be correct in all scenarios).*/ - Offset lastObject = new NullOffset(); // Abstract OffsetObject class, set to null object before initializing as RecordOffsetObject. + Offset lastObject = new NullOffset(); // Set to null object before initializing as RecordOffsetObject. + JsonObject lastObjectJo = JsonParser.parseString(lastObject.offsetToJSON()).getAsJsonObject(); long start = Instant.now().toEpochMilli(); // Starts measuring performance here. Measures how long it takes to process the whole recordOffsetObjectList. // This loop goes through all the records of the mock data in a single session. for (RecordOffset recordOffsetObject : recordOffsetObjectList) { + JsonObject recordOffsetObjectJo = JsonParser + .parseString(recordOffsetObject.offsetToJSON()) + .getAsJsonObject(); // Initializing syslogAvroWriter and lastObject. if (syslogAvroWriter == null && lastObject.isNull()) { try { - writableQueue.setQueueNamePrefix(recordOffsetObject.topic() + recordOffsetObject.partition()); + writableQueue + .setQueueNamePrefix(recordOffsetObjectJo.get("topic").getAsString() + recordOffsetObjectJo.get("partition").getAsString()); syslogFile = writableQueue.getNextWritableFile(); // The HDFS filename is only finalized when the AVRO-serialized file is finalized, because every Kafka-record added to the file is going to change the offset that is going to be used for the filename. syslogAvroWriter = new SyslogAvroWriter(syslogFile); lastObject = recordOffsetObject; + lastObjectJo = JsonParser.parseString(lastObject.offsetToJSON()).getAsJsonObject(); } catch (IOException ioException) { throw new IllegalArgumentException(ioException); @@ -200,21 +208,20 @@ public void accept(List recordOffsetObjectList) { else { try { if ( - lastObject.topic().equals(recordOffsetObject.topic()) - & lastObject.partition().equals(recordOffsetObject.partition()) + lastObjectJo.get("topic").getAsString().equals(recordOffsetObjectJo.get("topic").getAsString()) + & lastObjectJo.get("partition").getAsString().equals(recordOffsetObjectJo.get("partition").getAsString()) ) { // Records left to consume in the current partition. boolean fileCommitted = committedToHdfs( - syslogAvroWriter.getFileSize(), (RecordOffset) lastObject + syslogAvroWriter.getFileSize(), (RecordOffset) lastObject, lastObjectJo ); if (fileCommitted) { if (LOGGER.isDebugEnabled()) { LOGGER .debug( - "Target file size reached, file <{}> stored to <{}> in HDFS", - syslogFile.getName(), - lastObject.topic() + "/" + lastObject.partition() + "." - + lastObject.offset() + "Target file size reached, file <{}> stored to <{}> in HDFS", syslogFile + .getName(), + lastObjectJo.get("topic").getAsString() + "/" + lastObjectJo.get("partition").getAsString() + "." + lastObjectJo.get("offset").getAsString() ); } } @@ -231,11 +238,12 @@ public void accept(List recordOffsetObjectList) { else { // Previous partition was fully consumed. Commit file to HDFS and create a new AVRO-file. syslogAvroWriter.close(); - HDFSWrite writer = new HDFSWrite(config, (RecordOffset) lastObject); + HDFSWrite writer = new HDFSWrite(config, (RecordOffset) lastObject, lastObjectJo); writer.commit(syslogFile, epochMicros_last); // This part defines a new empty file to which the new AVRO-serialized records are stored until it again hits the 64M size limit. - writableQueue.setQueueNamePrefix(recordOffsetObject.topic() + recordOffsetObject.partition()); + writableQueue + .setQueueNamePrefix(recordOffsetObjectJo.get("topic").getAsString() + recordOffsetObjectJo.get("partition").getAsString()); syslogFile = writableQueue.getNextWritableFile(); syslogAvroWriter = new SyslogAvroWriter(syslogFile); } @@ -245,7 +253,7 @@ public void accept(List recordOffsetObjectList) { } } - byte[] byteArray = recordOffsetObject.record(); // loads the byte[] contained in recordOffsetObject.getRecord() to byteArray. + byte[] byteArray = recordOffsetObject.getRecord(); // loads the byte[] contained in recordOffsetObject.getRecord() to byteArray. batchBytes = batchBytes + byteArray.length; InputStream inputStream = new ByteArrayInputStream(byteArray); rfc5424Frame.load(inputStream); @@ -274,8 +282,8 @@ public void accept(List recordOffsetObjectList) { .setStream(rfc5424Frame.structuredData.getValue(teragrepStreamName).toString()) // Or is sourcetype/stream supposed to be rfc5424Frame.appName.toString() instead? .setHost(rfc5424Frame.hostname.toString()) .setInput(new String(source, StandardCharsets.UTF_8)) - .setPartition(recordOffsetObject.partition().toString()) - .setOffset(recordOffsetObject.offset()) + .setPartition(recordOffsetObjectJo.get("partition").getAsString()) + .setOffset(recordOffsetObjectJo.get("offset").getAsLong()) .setOrigin(new String(origin, StandardCharsets.UTF_8)) .build(); @@ -287,16 +295,15 @@ public void accept(List recordOffsetObjectList) { } // Check if there is still room in syslogAvroWriter for another syslogRecord. Commit syslogAvroWriter to HDFS if no room left, emptying it out in the process. boolean fileCommitted = committedToHdfs( - syslogAvroWriter.getFileSize() + capacity, (RecordOffset) lastObject + syslogAvroWriter.getFileSize() + capacity, (RecordOffset) lastObject, lastObjectJo ); if (fileCommitted) { if (LOGGER.isDebugEnabled()) { LOGGER .debug( - "Target file size reached, file <{}> stored to <{}> in HDFS", - syslogFile.getName(), - lastObject.topic() + "/" + lastObject.partition() + "." - + lastObject.offset() + "Target file size reached, file <{}> stored to <{}> in HDFS", syslogFile + .getName(), + lastObjectJo.get("topic").getAsString() + "/" + lastObjectJo.get("partition").getAsString() + "." + lastObjectJo.get("offset").getAsString() ); } } @@ -314,6 +321,7 @@ public void accept(List recordOffsetObjectList) { // Add syslogRecord to syslogAvroWriter which has room for new syslogRecord. syslogAvroWriter.write(syslogRecord); lastObject = recordOffsetObject; + lastObjectJo = JsonParser.parseString(lastObject.offsetToJSON()).getAsJsonObject(); } } catch (IOException e) { @@ -325,7 +333,7 @@ public void accept(List recordOffsetObjectList) { try { if (syslogAvroWriter != null && !lastObject.isNull()) { syslogAvroWriter.close(); - try (HDFSWrite writer = new HDFSWrite(config, (RecordOffset) lastObject)) { + try (HDFSWrite writer = new HDFSWrite(config, (RecordOffset) lastObject, lastObjectJo)) { writer.commit(syslogFile, epochMicros_last); // commits the final AVRO-file to HDFS. } } diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java index ac3d9c45..74d5a99b 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java @@ -45,6 +45,7 @@ */ package com.teragrep.cfe_39.consumers.kafka; +import com.google.gson.JsonObject; import com.teragrep.cfe_39.Config; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.*; @@ -68,7 +69,7 @@ public class HDFSWrite implements AutoCloseable { private final Configuration conf; private final String hdfsuri; - public HDFSWrite(Config config, RecordOffset lastObject) throws IOException { + public HDFSWrite(Config config, RecordOffset lastObject, JsonObject lastObjectJo) throws IOException { Properties readerKafkaProperties = config.getKafkaConsumerProperties(); this.useMockKafkaConsumer = Boolean @@ -81,8 +82,8 @@ public HDFSWrite(Config config, RecordOffset lastObject) throws IOException { /* The filepath should be something like hdfs:///opt/teragrep/cfe_39/srv/topic_name/0.12345 where 12345 is offset and 0 the partition. In other words the directory named topic_name holds files that are named and arranged based on partition and the partition's offset. Every partition has its own set of unique offset values. These values should be fetched from config and other input parameters (topic+partition+offset).*/ - path = config.getHdfsPath() + "/" + lastObject.topic(); - fileName = lastObject.partition() + "." + lastObject.offset(); // filename should be constructed from partition and offset. + path = config.getHdfsPath() + "/" + lastObjectJo.get("topic").getAsString(); + fileName = lastObjectJo.get("partition").getAsString() + "." + lastObjectJo.get("offset").getAsString(); // filename should be constructed from partition and offset. // ====== Init HDFS File System Object conf = new Configuration(); @@ -107,8 +108,8 @@ These values should be fetched from config and other input parameters (topic+par // Code for initializing the class for kerberized HDFS database usage. hdfsuri = config.getHdfsuri(); - path = config.getHdfsPath() + "/" + lastObject.topic(); - fileName = lastObject.partition() + "." + lastObject.offset(); + path = config.getHdfsPath() + "/" + lastObjectJo.get("topic").getAsString(); + fileName = lastObjectJo.get("partition").getAsString() + "." + lastObjectJo.get("offset").getAsString(); // set kerberos host and realm System.setProperty("java.security.krb5.realm", config.getKerberosRealm()); diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/NullOffset.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/NullOffset.java index dfb304e5..7f730df1 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/NullOffset.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/NullOffset.java @@ -54,22 +54,12 @@ public boolean isNull() { } @Override - public String topic() { - return "Not available"; - } - - @Override - public Integer partition() { - return 0; - } - - @Override - public Long offset() { - return 0L; + public byte[] getRecord() { + return new byte[0]; } @Override - public byte[] record() { - return new byte[0]; + public String offsetToJSON() { + return "{\"topic\":\"Not available\", \"partition\":\"0\", \"offset\":\"0\"}"; } } diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/Offset.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/Offset.java index 17b2fdb9..ada4c4fd 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/Offset.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/Offset.java @@ -49,11 +49,7 @@ public interface Offset { boolean isNull(); - String topic(); + byte[] getRecord(); - Integer partition(); - - Long offset(); - - byte[] record(); + String offsetToJSON(); } diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/RecordOffset.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/RecordOffset.java index e2136292..5f3369c0 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/RecordOffset.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/RecordOffset.java @@ -66,22 +66,16 @@ public boolean isNull() { } @Override - public String topic() { - return topic; - } - - @Override - public Integer partition() { - return partition; - } - - @Override - public Long offset() { - return offset; + public byte[] getRecord() { + return record; } @Override - public byte[] record() { - return record; + public String offsetToJSON() { + return String + .format( + "{\"topic\":\"%s\", \"partition\":\"%s\", \"offset\":\"%s\"}", this.topic, this.partition, + this.offset + ); } } diff --git a/src/test/java/com/teragrep/cfe_39/HdfsTest.java b/src/test/java/com/teragrep/cfe_39/HdfsTest.java index 49060700..2f99ea66 100644 --- a/src/test/java/com/teragrep/cfe_39/HdfsTest.java +++ b/src/test/java/com/teragrep/cfe_39/HdfsTest.java @@ -45,6 +45,8 @@ */ package com.teragrep.cfe_39; +import com.google.gson.JsonObject; +import com.google.gson.JsonParser; import com.teragrep.cfe_39.avro.SyslogRecord; import com.teragrep.cfe_39.consumers.kafka.HDFSWrite; import com.teragrep.cfe_39.consumers.kafka.HdfsDataIngestion; @@ -175,15 +177,15 @@ public void hdfsWriteTest() { lastRecord.getOffset(), null ); // Fetch input parameters from the lastRecord SyslogRecord-object. + JsonObject lastObjectJo = JsonParser.parseString(lastObject.offsetToJSON()).getAsJsonObject(); if (LOGGER.isDebugEnabled()) { LOGGER .debug( "\n" + "Last record in the " + syslogFile.getName() + " file:" + "\ntopic: " - + lastObject.topic() + "\npartition: " + lastObject.partition() - + "\noffset: " + lastObject.offset() + + lastObjectJo.get("topic").getAsString() + "\npartition: " + lastObjectJo.get("partition").getAsString() + "\noffset: " + lastObjectJo.get("offset").getAsString() ); } - try (HDFSWrite writer = new HDFSWrite(config, lastObject)) { + try (HDFSWrite writer = new HDFSWrite(config, lastObject, lastObjectJo)) { writer.commit(syslogFile, -1L); // commits the final AVRO-file to HDFS. } catch (IOException e) { From 8ce5ca472e1852a2c7d7fa1f7a5374b396103858 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Tue, 4 Jun 2024 10:22:26 +0300 Subject: [PATCH 084/146] Deleted QueueUtilities.java and moved the contained methods to WritableQueue.java --- .../consumers/kafka/queue/QueueUtilities.java | 100 ------------------ .../consumers/kafka/queue/WritableQueue.java | 49 ++++++++- 2 files changed, 46 insertions(+), 103 deletions(-) delete mode 100644 src/main/java/com/teragrep/cfe_39/consumers/kafka/queue/QueueUtilities.java diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/queue/QueueUtilities.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/queue/QueueUtilities.java deleted file mode 100644 index 834ebbe0..00000000 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/queue/QueueUtilities.java +++ /dev/null @@ -1,100 +0,0 @@ -/* - * HDFS Data Ingestion for PTH_06 use CFE-39 - * Copyright (C) 2021-2024 Suomen Kanuuna Oy - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU Affero General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU Affero General Public License for more details. - * - * You should have received a copy of the GNU Affero General Public License - * along with this program. If not, see . - * - * - * Additional permission under GNU Affero General Public License version 3 - * section 7 - * - * If you modify this Program, or any covered work, by linking or combining it - * with other code, such other code is not for that reason alone subject to any - * of the requirements of the GNU Affero GPL version 3 as long as this Program - * is the same Program as licensed from Suomen Kanuuna Oy without any additional - * modifications. - * - * Supplemented terms under GNU Affero General Public License version 3 - * section 7 - * - * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified - * versions must be marked as "Modified version of" The Program. - * - * Names of the licensors and authors may not be used for publicity purposes. - * - * No rights are granted for use of trade names, trademarks, or service marks - * which are in The Program if any. - * - * Licensee must indemnify licensors and authors for any liability that these - * contractual assumptions impose on licensors and authors. - * - * To the extent this program is licensed as part of the Commercial versions of - * Teragrep, the applicable Commercial License may apply to this file if you as - * a licensee so wish it. - */ -package com.teragrep.cfe_39.consumers.kafka.queue; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.nio.file.Files; -import java.nio.file.Path; -import java.nio.file.attribute.BasicFileAttributes; -import java.util.function.BiPredicate; -import java.util.function.ToLongFunction; - -class QueueUtilities { - - private static final Logger LOGGER = LoggerFactory.getLogger(QueueUtilities.class); - - static BiPredicate getFileMatcher(String queueNamePrefix) { - return (path, basicFileAttributes) -> { - if (!path.getFileName().toString().startsWith(queueNamePrefix)) { - return false; - } - else if (path.getFileName().toString().endsWith(".state")) { - return false; - } - else if (!basicFileAttributes.isRegularFile()) { - return false; - } - else { - LOGGER.trace("getFileMatcher returning: <{}>", path); - return true; - } - }; - } - - static void accessCheck(Path queueDirectory) { - if (!Files.isDirectory(queueDirectory)) { - throw new IllegalArgumentException("Provided path is not a " + "directory <[" + queueDirectory + "]>"); - } - - if (!Files.isWritable(queueDirectory)) { - throw new IllegalArgumentException("Provided path is not " + "writeable <[" + queueDirectory + "]>"); - } - } - - static ToLongFunction getPathToSequenceNumberFunction() { - return path -> { - String pathString = path.toString(); - - int dotPosition = pathString.lastIndexOf('.'); - - String sequenceNumberString = pathString.substring(dotPosition + 1); - - return Long.parseLong(sequenceNumberString); - }; - } -} diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/queue/WritableQueue.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/queue/WritableQueue.java index 3df56a93..6b2de85a 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/queue/WritableQueue.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/queue/WritableQueue.java @@ -45,6 +45,9 @@ */ package com.teragrep.cfe_39.consumers.kafka.queue; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.io.File; import java.io.IOException; import java.io.UncheckedIOException; @@ -52,26 +55,36 @@ import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; +import java.nio.file.attribute.BasicFileAttributes; +import java.util.function.BiPredicate; +import java.util.function.ToLongFunction; import java.util.stream.Stream; public class WritableQueue { + private static final Logger LOGGER = LoggerFactory.getLogger(WritableQueue.class); + private final Path queueDirectory; private String queueNamePrefix; public WritableQueue(String queueDirectory) { this.queueDirectory = Paths.get(queueDirectory); this.queueNamePrefix = ""; - QueueUtilities.accessCheck(this.queueDirectory); + if (!Files.isDirectory(this.queueDirectory)) { + throw new IllegalArgumentException("Provided path is not a " + "directory <[" + queueDirectory + "]>"); + } + if (!Files.isWritable(this.queueDirectory)) { + throw new IllegalArgumentException("Provided path is not " + "writeable <[" + queueDirectory + "]>"); + } } private File getNextWritableFilename() throws IOException { try ( - Stream files = Files.find(queueDirectory, 1, QueueUtilities.getFileMatcher(queueNamePrefix), FileVisitOption.FOLLOW_LINKS) + Stream files = Files.find(queueDirectory, 1, getFileMatcher(queueNamePrefix), FileVisitOption.FOLLOW_LINKS) ) { - long sequenceNumber = files.mapToLong(QueueUtilities.getPathToSequenceNumberFunction()).max().orElse(0); + long sequenceNumber = files.mapToLong(getPathToSequenceNumberFunction()).max().orElse(0); long nextSequenceNumber = sequenceNumber + 1; @@ -98,4 +111,34 @@ public File getNextWritableFile() throws IOException { public void setQueueNamePrefix(String a) { this.queueNamePrefix = a; } + + private BiPredicate getFileMatcher(String queueNamePrefix) { + return (path, basicFileAttributes) -> { + if (!path.getFileName().toString().startsWith(queueNamePrefix)) { + return false; + } + else if (path.getFileName().toString().endsWith(".state")) { + return false; + } + else if (!basicFileAttributes.isRegularFile()) { + return false; + } + else { + LOGGER.trace("getFileMatcher returning: <{}>", path); + return true; + } + }; + } + + private ToLongFunction getPathToSequenceNumberFunction() { + return path -> { + String pathString = path.toString(); + + int dotPosition = pathString.lastIndexOf('.'); + + String sequenceNumberString = pathString.substring(dotPosition + 1); + + return Long.parseLong(sequenceNumberString); + }; + } } From 65342c2e865912f981dfe82f470cab6884f72cdb Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Tue, 4 Jun 2024 15:42:27 +0300 Subject: [PATCH 085/146] Refactoring tests according to the code review. WIP --- .../com/teragrep/cfe_39/CombinedFullTest.java | 667 +++++++++--------- .../java/com/teragrep/cfe_39/HdfsTest.java | 499 +++++++------ .../teragrep/cfe_39/KafkaConsumerTest.java | 407 +++++------ 3 files changed, 771 insertions(+), 802 deletions(-) diff --git a/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java b/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java index 108f174b..466c179c 100644 --- a/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java +++ b/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java @@ -53,152 +53,129 @@ import org.apache.hadoop.fs.*; import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.MiniDFSCluster; -import org.junit.jupiter.api.AfterAll; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.*; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.fs.Path; import java.io.File; -import java.io.IOException; import java.net.URI; import java.nio.file.Files; import java.util.*; import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; + public class CombinedFullTest { private static final Logger LOGGER = LoggerFactory.getLogger(CombinedFullTest.class); private static MiniDFSCluster hdfsCluster; private static File baseDir; private static Config config; + private FileSystem fs; // Start minicluster and initialize config. - @BeforeAll - public static void startMiniCluster() throws IOException, InterruptedException { - config = null; - try { + @BeforeEach + public void startMiniCluster() { + assertDoesNotThrow(() -> { config = new Config(); - } - catch (IOException e) { - LOGGER.error("Can't load config: {}", e.toString()); - Assertions.fail(); - } - catch (IllegalArgumentException e) { - LOGGER.error("Got invalid config: {}", e.toString()); - Assertions.fail(); - } - // Create a HDFS miniCluster - baseDir = Files.createTempDirectory("test_hdfs").toFile().getAbsoluteFile(); - Configuration conf = new Configuration(); - conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, baseDir.getAbsolutePath()); - MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf); - hdfsCluster = builder.build(); - String hdfsURI = "hdfs://localhost:" + hdfsCluster.getNameNodePort() + "/"; - config.setHdfsuri(hdfsURI); - DistributedFileSystem fileSystem = hdfsCluster.getFileSystem(); + // Create a HDFS miniCluster + baseDir = Files.createTempDirectory("test_hdfs").toFile().getAbsoluteFile(); + Configuration conf = new Configuration(); + conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, baseDir.getAbsolutePath()); + MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf); + hdfsCluster = builder.build(); + String hdfsURI = "hdfs://localhost:" + hdfsCluster.getNameNodePort() + "/"; + config.setHdfsuri(hdfsURI); + DistributedFileSystem fileSystem = hdfsCluster.getFileSystem(); + + // ====== Init HDFS File System Object + Configuration fsConf = new Configuration(); + // Set FileSystem URI + fsConf.set("fs.defaultFS", hdfsURI); + // Because of Maven + fsConf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); + fsConf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); + // Set HADOOP user + System.setProperty("HADOOP_USER_NAME", "hdfs"); + System.setProperty("hadoop.home.dir", "/"); + String path = config.getHdfsPath() + "/" + "testConsumerTopic"; // "hdfs:///opt/teragrep/cfe_39/srv/testConsumerTopic" + fs = FileSystem.get(URI.create(hdfsURI), fsConf); + //==== Create directory if not exists + Path workingDir = fs.getWorkingDirectory(); + // Sets the directory where the data should be stored, if the directory doesn't exist then it's created. + Path newDirectoryPath = new Path(path); + if (!fs.exists(newDirectoryPath)) { + // Create new Directory + fs.mkdirs(newDirectoryPath); + LOGGER.debug("Path {} created.", path); + } + }); } // Teardown the minicluster - @AfterAll - public static void teardownMiniCluster() { + @AfterEach + public void teardownMiniCluster() { + assertDoesNotThrow(() -> { + fs.close(); + }); hdfsCluster.shutdown(); FileUtil.fullyDelete(baseDir); } @Test - public void kafkaAndAvroFullTest() throws InterruptedException, IOException { - insertMockFiles(); // Maybe add modifier that allows adding or not adding the mock file. - config.setMaximumFileSize(3000); // 10 loops (140 records) are in use at the moment, and that is sized at 36,102 bytes. - HdfsDataIngestion hdfsDataIngestion = new HdfsDataIngestion(config); - Thread.sleep(10000); - hdfsDataIngestion.run(); - /* The avro files should be committed to HDFS now. Check the committed files for any errors. - There should be 20 files, 10 partitions with each having 2 files assigned to them. - hdfsReadCheck(); does not work properly if pruning is enabled and prune offset is set too low, which causes the records to be pruned from the database.*/ - if (config.getPruneOffset() == 157784760000L) { - try { - hdfsReadCheck(); - } - catch (IOException e) { - throw new RuntimeException(e); - } - } - + public void kafkaAndAvroFullTest() { + // Empty HDFS database, 140 records in mock kafka consumer ready for ingestion. All 14 records for each 10 topic partitions are stored in a single avro-file per partition. + assertDoesNotThrow(() -> { + Assertions.assertTrue(config.getPruneOffset() >= 300000L); // Fails the test if the config is not correct. + config.setMaximumFileSize(30000); + HdfsDataIngestion hdfsDataIngestion = new HdfsDataIngestion(config); + Thread.sleep(10000); + hdfsDataIngestion.run(); + // Assert that the kafka records were ingested correctly and the database holds the correct 140 records. + hdfsReadCheck(); + }); } - @Test - public void hdfsPruneTest() throws IOException { - // Check that the files were properly written to HDFS with a read test. - String hdfsuri = config.getHdfsuri(); + // TODO: Make a separate test out of this. + // 1. Insert the pre-made files into the database using insertMockFiles(). + // 2. set their timestamp to something that will be pruned. + // 3. Run HdfsDataIngestion like in kafkaAndAvroFullTest(). + // 4. Assert that only the expected files are present in the HDFS database. + public void hdfsPruneTest() { String path = config.getHdfsPath() + "/" + "testConsumerTopic"; - // ====== Init HDFS File System Object - Configuration conf = new Configuration(); - // Set FileSystem URI - conf.set("fs.defaultFS", hdfsuri); - // Because of Maven - conf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); - conf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); - // Set HADOOP user - System.setProperty("HADOOP_USER_NAME", "hdfs"); - System.setProperty("hadoop.home.dir", "/"); //Get the filesystem - HDFS - FileSystem fs = FileSystem.get(URI.create(hdfsuri), conf); - - //==== Create directory if not exists - Path workingDir = fs.getWorkingDirectory(); - Path newDirectoryPath = new Path(path); - if (!fs.exists(newDirectoryPath)) { - // Create new Directory - fs.mkdirs(newDirectoryPath); - LOGGER.info("Path {} created.", path); - } - - /* Use either HDFS-file modification timestamps or avro-mapred for pruning. - - The records are in this AVRO format: - {"timestamp": 1650872092240000, "message": "25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn audit says hi!]", "directory": "jla02logger", "stream": "test:jla02logger:0", "host": "jla-02.default", "input": "imrelp:cfe-06-0.cfe-06.default:", "partition": "8", "offset": 8, "origin": "jla-02.default"} - Query handler must be implemented in a way that the AVRO files are first opened, then processed to syslog format and then sent to the query requester. The records are processed/filtered based on the given query conditions using MapReduce to make the code capable of processing the vast amounts of records that are expected. - MapReduce functionalities of the Hadoop cluster: https://hadoop.apache.org/docs/stable/hadoop-mapreduce-client/hadoop-mapreduce-client-core/MapReduceTutorial.html - Avro side of documentations for MapReduce: https://avro.apache.org/docs/1.11.1/mapreduce-guide/ - - Another method for pruning aside using avro-mapred is to use modification timestamp of the avro-file stored in HDFS: - fs.setTimes(new Path(path+"/"+0.8), Long.parseUnsignedLong("1675930598000"), -1); - where mtime is modification time and atime is access time. -1 as input parameter leaves the original atime/mtime value as is.*/ - FileStatus[] fileStatuses = fs.listStatus(new Path(newDirectoryPath + "/")); - long count = Arrays.stream(fileStatuses).count(); - if (count != 0) { - if (config.getPruneOffset() != 157784760000L) { - Assertions.fail("There are files available in the database when there should be none."); - } - boolean delete = false; - for (FileStatus a : fileStatuses) { - // Delete old files - if (a.getModificationTime() < System.currentTimeMillis()) { - delete = fs.delete(a.getPath(), true); - Assertions.assertTrue(delete); - LOGGER.info("Deleted file {}", a.getPath()); - } - } - Assertions.assertTrue(delete); - LOGGER.info("All files were pruned properly."); - } - else { - if (config.getPruneOffset() == 157784760000L) { - Assertions.fail("There were no files available in the database when there should be."); - } - LOGGER.info("No files available as they were pruned properly already!"); - } + assertDoesNotThrow(() -> { - fs.close(); + Path workingDir = fs.getWorkingDirectory(); + Path newDirectoryPath = new Path(path); + Assertions.assertTrue(fs.exists(newDirectoryPath)); + + /* Use HDFS-file modification timestamps for pruning. + fs.setTimes(new Path(path+"/"+0.8), Long.parseUnsignedLong("1675930598000"), -1); + where mtime is modification time and atime is access time. -1 as input parameter leaves the original atime/mtime value as is.*/ + FileStatus[] fileStatuses = fs.listStatus(new Path(newDirectoryPath + "/")); + long count = Arrays.stream(fileStatuses).count(); + Assertions.assertTrue(count > 0); + + for (FileStatus a : fileStatuses) { + // Delete old files + if (a.getModificationTime() < System.currentTimeMillis()) { + boolean deleted = fs.delete(a.getPath(), true); + Assertions.assertTrue(deleted); + LOGGER.info("Deleted file {}", a.getPath()); + LOGGER.debug("Timestamp was {}", a.getModificationTime()); // example: 1717582386630 milliseconds which translates to Wednesday, June 5, 2024 1:13:06.630 PM GMT+03:00 DST + } + } + LOGGER.info("All files were pruned properly."); + }); } - public void hdfsReadCheck() throws IOException { - // Check that the files were properly written to HDFS with a read test. + // This method checks that the expected files with expected contents are present in the HDFS database. + public void hdfsReadCheck() { + // Check that the files were properly written to HDFS. String hdfsuri = config.getHdfsuri(); String path = config.getHdfsPath() + "/" + "testConsumerTopic"; @@ -213,247 +190,281 @@ public void hdfsReadCheck() throws IOException { System.setProperty("HADOOP_USER_NAME", "hdfs"); System.setProperty("hadoop.home.dir", "/"); //Get the filesystem - HDFS - FileSystem fs = FileSystem.get(URI.create(hdfsuri), conf); - - //==== Create directory if not exists - Path workingDir = fs.getWorkingDirectory(); - Path newDirectoryPath = new Path(path); - if (!fs.exists(newDirectoryPath)) { - // Create new Directory - fs.mkdirs(newDirectoryPath); - LOGGER.info("Path {} created.", path); - } - - /* This is the HDFS write path for the files: - Path hdfswritepath = new Path(newDirectoryPath + "/" + fileName); where newDirectoryPath is config.getHdfsPath() + "/" + lastObject.topic; and filename is lastObject.partition+"."+lastObject.offset; - - Create the list of files to read from HDFS. Test setup is created so each of the 0-9 partitions will have 2 files with offsets of 8 and 13.*/ - List filenameList = new ArrayList<>(); - for (int i = 0; i <= 9; i++) { - filenameList.add(i + "." + 9); - filenameList.add(i + "." + 13); - } - int looper = 0; - int partitionCounter = 0; - for (String fileName : filenameList) { - //==== Read files - LOGGER.info("Read file into hdfs"); - //Create a path - Path hdfsreadpath = new Path(newDirectoryPath + "/" + fileName); // The path should be the same that was used in writing the file to HDFS. - //Init input stream - FSDataInputStream inputStream = fs.open(hdfsreadpath); - //The data is in AVRO-format, so it can't be read as a string. - DataFileStream reader = new DataFileStream<>( - inputStream, - new SpecificDatumReader<>(SyslogRecord.class) - ); - SyslogRecord record = null; - LOGGER.info("\nReading records from file {}:", hdfsreadpath.toString()); - while (reader.hasNext()) { + assertDoesNotThrow(() -> { + FileSystem fs = FileSystem.get(URI.create(hdfsuri), conf); + + Path workingDir = fs.getWorkingDirectory(); + Path newDirectoryPath = new Path(path); + Assertions.assertTrue(fs.exists(newDirectoryPath)); + + /* This is the HDFS write path for the files: + Path hdfswritepath = new Path(newDirectoryPath + "/" + fileName); where newDirectoryPath is config.getHdfsPath() + "/" + lastObject.topic; and filename is lastObject.partition+"."+lastObject.offset; + + Create the list of files to read from HDFS. Test setup is created so each of the 0-9 partitions will have 1 file with offset of 13.*/ + List filenameList = new ArrayList<>(); + for (int i = 0; i <= 9; i++) { + filenameList.add(i + "." + 13); + } + FileStatus[] fileStatuses = fs.listStatus(newDirectoryPath); + Assertions.assertEquals(filenameList.size(), fileStatuses.length); + for (FileStatus fileStatus : fileStatuses) { + Assertions.assertTrue(filenameList.contains(fileStatus.getPath().getName())); + } + LOGGER.debug("All expected files present in HDFS."); + + int partitionCounter = 0; + for (String fileName : filenameList) { + //==== Read files + LOGGER.info("Read file into hdfs"); + //Create a path + Path hdfsreadpath = new Path(newDirectoryPath + "/" + fileName); // The path should be the same that was used in writing the file to HDFS. + //Init input stream + FSDataInputStream inputStream = fs.open(hdfsreadpath); + //The data is in AVRO-format, so it can't be read as a string. + DataFileStream reader = new DataFileStream<>( + inputStream, + new SpecificDatumReader<>(SyslogRecord.class) + ); + SyslogRecord record = null; + LOGGER.info("\nReading records from file {}:", hdfsreadpath); + + Assertions.assertTrue(reader.hasNext()); record = reader.next(record); if (LOGGER.isDebugEnabled()) { LOGGER.debug(record.toString()); } - // Assert records here like it is done in KafkaConsumerTest.avroReader(). - if (looper <= 0) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872090804000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 0, \"origin\": \"jla-02.default\", \"payload\": \"[WARN] 2022-04-25 07:34:50,804 com.teragrep.jla_02.Log4j Log - Log4j warn says hi!\"}", - record.toString() - ); - looper++; - } - else if (looper == 1) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872090806000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 1, \"origin\": \"jla-02.default\", \"payload\": \"[ERROR] 2022-04-25 07:34:50,806 com.teragrep.jla_02.Log4j Log - Log4j error says hi!\"}", - record.toString() - ); - looper++; + Assertions + .assertEquals( + "{\"timestamp\": 1650872090804000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 0, \"origin\": \"jla-02.default\", \"payload\": \"[WARN] 2022-04-25 07:34:50,804 com.teragrep.jla_02.Log4j Log - Log4j warn says hi!\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug(record.toString()); } - else if (looper == 2) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872090822000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 2, \"origin\": \"jla-02\", \"payload\": \"470647 [Thread-3] INFO com.teragrep.jla_02.Logback Daily - Logback-daily says hi.\"}", - record.toString() - ); - looper++; + Assertions + .assertEquals( + "{\"timestamp\": 1650872090806000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 1, \"origin\": \"jla-02.default\", \"payload\": \"[ERROR] 2022-04-25 07:34:50,806 com.teragrep.jla_02.Log4j Log - Log4j error says hi!\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug(record.toString()); } - else if (looper == 3) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872090822000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 3, \"origin\": \"jla-02\", \"payload\": \"470646 [Thread-3] INFO com.teragrep.jla_02.Logback Audit - Logback-audit says hi.\"}", - record.toString() - ); - looper++; + Assertions + .assertEquals( + "{\"timestamp\": 1650872090822000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 2, \"origin\": \"jla-02\", \"payload\": \"470647 [Thread-3] INFO com.teragrep.jla_02.Logback Daily - Logback-daily says hi.\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug(record.toString()); } - else if (looper == 4) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872090822000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 4, \"origin\": \"jla-02\", \"payload\": \"470647 [Thread-3] INFO com.teragrep.jla_02.Logback Metric - Logback-metric says hi.\"}", - record.toString() - ); - looper++; + Assertions + .assertEquals( + "{\"timestamp\": 1650872090822000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 3, \"origin\": \"jla-02\", \"payload\": \"470646 [Thread-3] INFO com.teragrep.jla_02.Logback Audit - Logback-audit says hi.\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug(record.toString()); } - else if (looper == 5) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872092238000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 5, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.238 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info audit says hi!]\"}", - record.toString() - ); - looper++; + Assertions + .assertEquals( + "{\"timestamp\": 1650872090822000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 4, \"origin\": \"jla-02\", \"payload\": \"470647 [Thread-3] INFO com.teragrep.jla_02.Logback Metric - Logback-metric says hi.\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug(record.toString()); } - else if (looper == 6) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872092239000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 6, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info daily says hi!]\"}", - record.toString() - ); - looper++; + Assertions + .assertEquals( + "{\"timestamp\": 1650872092238000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 5, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.238 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info audit says hi!]\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug(record.toString()); } - else if (looper == 7) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872092239000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 7, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info metric says hi!]\"}", - record.toString() - ); - looper++; + Assertions + .assertEquals( + "{\"timestamp\": 1650872092239000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 6, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info daily says hi!]\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug(record.toString()); } - else if (looper == 8) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872092240000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 8, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn audit says hi!]\"}", - record.toString() - ); - looper++; + Assertions + .assertEquals( + "{\"timestamp\": 1650872092239000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 7, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info metric says hi!]\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug(record.toString()); } - else if (looper == 9) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872092240000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 9, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn daily says hi!]\"}", - record.toString() - ); - looper++; + Assertions + .assertEquals( + "{\"timestamp\": 1650872092240000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 8, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn audit says hi!]\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug(record.toString()); } - else if (looper == 10) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872092241000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 10, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.241 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn metric says hi!]\"}", - record.toString() - ); - looper++; + Assertions + .assertEquals( + "{\"timestamp\": 1650872092240000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 9, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn daily says hi!]\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug(record.toString()); } - else if (looper == 11) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872092241000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 11, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.241 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error audit says hi!]\"}", - record.toString() - ); - looper++; + Assertions + .assertEquals( + "{\"timestamp\": 1650872092241000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 10, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.241 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn metric says hi!]\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug(record.toString()); } - else if (looper == 12) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872092242000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 12, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.242 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error daily says hi!]\"}", - record.toString() - ); - looper++; + Assertions + .assertEquals( + "{\"timestamp\": 1650872092241000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 11, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.241 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error audit says hi!]\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug(record.toString()); } - else { - Assertions - .assertEquals( - "{\"timestamp\": 1650872092243000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 13, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.243 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error metric says hi!]\"}", - record.toString() - ); - looper = 0; - LOGGER.info("Partition {} passed assertions.", partitionCounter); - partitionCounter++; + Assertions + .assertEquals( + "{\"timestamp\": 1650872092242000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 12, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.242 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error daily says hi!]\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug(record.toString()); } + Assertions + .assertEquals( + "{\"timestamp\": 1650872092243000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 13, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.243 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error metric says hi!]\"}", + record.toString() + ); + Assertions.assertFalse(reader.hasNext()); + LOGGER.info("Partition {} passed assertions.", partitionCounter); + partitionCounter++; + inputStream.close(); } - inputStream.close(); - } - fs.close(); + Assertions.assertEquals(10, partitionCounter); + }); } - private void insertMockFiles() throws IOException { + // Inserts pre-made avro-files to HDFS, which are normally generated during data ingestion from mock kafka consumer. + private void insertMockFiles(long a, long b) { String path = config.getHdfsPath() + "/" + "testConsumerTopic"; // "hdfs:///opt/teragrep/cfe_39/srv/testConsumerTopic" - String hdfsURI = config.getHdfsuri(); - // ====== Init HDFS File System Object - Configuration conf = new Configuration(); - // Set FileSystem URI - conf.set("fs.defaultFS", hdfsURI); - // Because of Maven - conf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); - conf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); - // Set HADOOP user - System.setProperty("HADOOP_USER_NAME", "hdfs"); - System.setProperty("hadoop.home.dir", "/"); //Get the filesystem - HDFS - FileSystem fs = FileSystem.get(URI.create(hdfsURI), conf); - - //==== Create directory if not exists - Path workingDir = fs.getWorkingDirectory(); - // Sets the directory where the data should be stored, if the directory doesn't exist then it's created. - Path newDirectoryPath = new Path(path); - if (!fs.exists(newDirectoryPath)) { - // Create new Directory - fs.mkdirs(newDirectoryPath); - LOGGER.debug("Path {} created.", path); - } - - String dir = System.getProperty("user.dir") + "/src/test/java/com/teragrep/cfe_39/mockHdfsFiles"; - Set listOfFiles = Stream - .of(Objects.requireNonNull(new File(dir).listFiles())) - .filter(file -> !file.isDirectory()) - .map(File::getName) - .collect(Collectors.toSet()); - // Loop through all the avro files - for (String fileName : listOfFiles) { - String pathname = dir + "/" + fileName; - File avroFile = new File(pathname); - //==== Write file - LOGGER.debug("Begin Write file into hdfs"); - //Create a path - Path hdfswritepath = new Path(newDirectoryPath + "/" + avroFile.getName()); // filename should be set according to the requirements: 0.12345 where 0 is Kafka partition and 12345 is Kafka offset. - if (fs.exists(hdfswritepath)) { - throw new RuntimeException("File " + avroFile.getName() + " already exists"); + assertDoesNotThrow(() -> { + + //==== Create directory if not exists + Path workingDir = fs.getWorkingDirectory(); + // Sets the directory where the data should be stored, if the directory doesn't exist then it's created. + Path newDirectoryPath = new Path(path); + if (!fs.exists(newDirectoryPath)) { + // Create new Directory + fs.mkdirs(newDirectoryPath); + LOGGER.debug("Path {} created.", path); } - Path readPath = new Path(avroFile.getPath()); - // Add conditions if file filtering is required for tests. - fs.copyFromLocalFile(readPath, hdfswritepath); - LOGGER.debug("End Write file into hdfs"); - if (LOGGER.isDebugEnabled()) { - LOGGER.debug("\nFile committed to HDFS, file writepath should be: {}\n", hdfswritepath.toString()); + + String dir = System.getProperty("user.dir") + "/src/test/java/com/teragrep/cfe_39/mockHdfsFiles"; + Set listOfFiles = Stream + .of(Objects.requireNonNull(new File(dir).listFiles())) + .filter(file -> !file.isDirectory()) + .map(File::getName) + .collect(Collectors.toSet()); + // Loop through all the avro files + for (String fileName : listOfFiles) { + String pathname = dir + "/" + fileName; + File avroFile = new File(pathname); + //==== Write file + LOGGER.debug("Begin Write file into hdfs"); + //Create a path + Path hdfswritepath = new Path(newDirectoryPath + "/" + avroFile.getName()); // filename should be set according to the requirements: 0.12345 where 0 is Kafka partition and 12345 is Kafka offset. + if (fs.exists(hdfswritepath)) { + Assertions.fail("File " + avroFile.getName() + " already exists"); + } + Path readPath = new Path(avroFile.getPath()); + // Add conditions if file filtering is required for tests. + fs.copyFromLocalFile(readPath, hdfswritepath); + // Set a/b to something like 157784760000 to trigger pruning. + if (Objects.equals(hdfswritepath.toString(), "hdfs:/opt/teragrep/cfe_39/srv/testConsumerTopic/0.9")) { + fs.setTimes(hdfswritepath, a, -1); + } else if (Objects.equals(hdfswritepath.toString(), "hdfs:/opt/teragrep/cfe_39/srv/testConsumerTopic/0.13")) { + fs.setTimes(hdfswritepath, b, -1); + } + LOGGER.debug("End Write file into hdfs"); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug("\nFile committed to HDFS, file writepath should be: {}\n", hdfswritepath.toString()); + } } - } - fs.close(); + }); } } diff --git a/src/test/java/com/teragrep/cfe_39/HdfsTest.java b/src/test/java/com/teragrep/cfe_39/HdfsTest.java index 2f99ea66..5ec26806 100644 --- a/src/test/java/com/teragrep/cfe_39/HdfsTest.java +++ b/src/test/java/com/teragrep/cfe_39/HdfsTest.java @@ -62,15 +62,16 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.MiniDFSCluster; -import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.*; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.File; -import java.io.IOException; import java.net.URI; import java.nio.file.Files; +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; + public class HdfsTest { private static final Logger LOGGER = LoggerFactory.getLogger(HdfsTest.class); @@ -80,52 +81,44 @@ public class HdfsTest { private static Config config; // Generate AVRO-files for testing the HDFS writes. - // @BeforeAll - public static void generateTestData() throws IOException, InterruptedException { - config = null; - try { + @Disabled + @BeforeAll + public static void generateTestData() { + assertDoesNotThrow(() -> { config = new Config(); - } - catch (IOException e) { - LOGGER.error("Can't load config: " + e); - System.exit(1); - } - catch (IllegalArgumentException e) { - LOGGER.error("Got invalid config: " + e); - System.exit(1); - } - startMiniCluster(); - config.setMaximumFileSize(3000); // 10 loops (140 records) are in use at the moment, and that is sized at 36,102 bytes. - HdfsDataIngestion hdfsDataIngestion = new HdfsDataIngestion(config); - hdfsDataIngestion.run(); + startMiniCluster(); + config.setMaximumFileSize(3000); // 10 loops (140 records) are in use at the moment, and that is sized at 36,102 bytes. + HdfsDataIngestion hdfsDataIngestion = new HdfsDataIngestion(config); + hdfsDataIngestion.run(); + }); } - public static void startMiniCluster() throws IOException { - // Create a HDFS miniCluster - baseDir = Files.createTempDirectory("test_hdfs").toFile().getAbsoluteFile(); - Configuration conf = new Configuration(); - conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, baseDir.getAbsolutePath()); - MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf); - hdfsCluster = builder.build(); - String hdfsURI = "hdfs://localhost:" + hdfsCluster.getNameNodePort() + "/"; - LOGGER.debug("hdfsURI: " + hdfsURI); - config.setHdfsuri(hdfsURI); - DistributedFileSystem fileSystem = hdfsCluster.getFileSystem(); + public static void startMiniCluster() { + assertDoesNotThrow(() -> { + // Create a HDFS miniCluster + baseDir = Files.createTempDirectory("test_hdfs").toFile().getAbsoluteFile(); + Configuration conf = new Configuration(); + conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, baseDir.getAbsolutePath()); + MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf); + hdfsCluster = builder.build(); + String hdfsURI = "hdfs://localhost:" + hdfsCluster.getNameNodePort() + "/"; + LOGGER.debug("hdfsURI: " + hdfsURI); + config.setHdfsuri(hdfsURI); + DistributedFileSystem fileSystem = hdfsCluster.getFileSystem(); + }); } // Delete the generated AVRO-files. - // @AfterAll - public static void deleteTestData() throws IOException { + @Disabled + @AfterAll + public static void deleteTestData() { Path queueDirectory = new Path(config.getQueueDirectory()); // Paths.get(config.getQueueDirectory()); for (int j = 0; j <= 9; j++) { for (int i = 1; i <= 2; i++) { File syslogFile = new File(queueDirectory.toUri() + File.separator + "testConsumerTopic" + j + "." + i); - try { - boolean result = Files.deleteIfExists(syslogFile.toPath()); //surround it in try catch block - } - catch (IOException e) { - throw new RuntimeException(e); - } + assertDoesNotThrow(() -> { + boolean result = Files.deleteIfExists(syslogFile.toPath()); + }); } } closeMiniCluster(); @@ -137,81 +130,61 @@ public static void closeMiniCluster() { FileUtil.fullyDelete(baseDir); } - // @Test - public void miniClusterDebugging() throws InterruptedException, IOException { - startMiniCluster(); - closeMiniCluster(); - } - - // @Test + @Disabled + @Test public void hdfsWriteTest() { + startMiniCluster(); + assertDoesNotThrow(() -> { + Path queueDirectory = new Path(config.getQueueDirectory()); + for (int j = 0; j <= 9; j++) { + for (int i = 1; i <= 2; i++) { + File syslogFile = new File( + queueDirectory.toUri() + File.separator + "testConsumerTopic" + j + "." + i + ); - try { - startMiniCluster(); - } - catch (IOException e) { - throw new RuntimeException(e); - } - - Path queueDirectory = new Path(config.getQueueDirectory()); - for (int j = 0; j <= 9; j++) { - for (int i = 1; i <= 2; i++) { - File syslogFile = new File(queueDirectory.toUri() + File.separator + "testConsumerTopic" + j + "." + i); - - // generate lastObject from the last record in the file in this test - DatumReader userDatumReader = new SpecificDatumReader<>(SyslogRecord.class); - SyslogRecord lastRecord = null; - try (DataFileReader dataFileReader = new DataFileReader<>(syslogFile, userDatumReader)) { - while (dataFileReader.hasNext()) { - lastRecord = dataFileReader.next(lastRecord); + // generate lastObject from the last record in the file in this test + DatumReader userDatumReader = new SpecificDatumReader<>(SyslogRecord.class); + SyslogRecord lastRecord = null; + try ( + DataFileReader dataFileReader = new DataFileReader<>( + syslogFile, + userDatumReader + ) + ) { + while (dataFileReader.hasNext()) { + lastRecord = dataFileReader.next(lastRecord); + } } - } - catch (IOException e) { - throw new RuntimeException(e); - } - assert lastRecord != null; - RecordOffset lastObject = new RecordOffset( - "testConsumerTopic", - Integer.parseInt(lastRecord.getPartition().toString()), - lastRecord.getOffset(), - null - ); // Fetch input parameters from the lastRecord SyslogRecord-object. - JsonObject lastObjectJo = JsonParser.parseString(lastObject.offsetToJSON()).getAsJsonObject(); - if (LOGGER.isDebugEnabled()) { - LOGGER - .debug( - "\n" + "Last record in the " + syslogFile.getName() + " file:" + "\ntopic: " - + lastObjectJo.get("topic").getAsString() + "\npartition: " + lastObjectJo.get("partition").getAsString() + "\noffset: " + lastObjectJo.get("offset").getAsString() - ); - } - try (HDFSWrite writer = new HDFSWrite(config, lastObject, lastObjectJo)) { + assert lastRecord != null; + RecordOffset lastObject = new RecordOffset( + "testConsumerTopic", + Integer.parseInt(lastRecord.getPartition().toString()), + lastRecord.getOffset(), + null + ); // Fetch input parameters from the lastRecord SyslogRecord-object. + JsonObject lastObjectJo = JsonParser.parseString(lastObject.offsetToJSON()).getAsJsonObject(); + if (LOGGER.isDebugEnabled()) { + LOGGER + .debug( + "\n" + "Last record in the " + syslogFile.getName() + " file:" + "\ntopic: " + + lastObjectJo.get("topic").getAsString() + "\npartition: " + lastObjectJo.get("partition").getAsString() + "\noffset: " + lastObjectJo.get("offset").getAsString() + ); + } + HDFSWrite writer = new HDFSWrite(config, lastObject, lastObjectJo); writer.commit(syslogFile, -1L); // commits the final AVRO-file to HDFS. - } - catch (IOException e) { - throw new RuntimeException(e); - } - - // Check that the file was stored to HDFS properly. - try { Thread.sleep(1000); hdfsReadCheck( "testConsumerTopic", Integer.parseInt(lastRecord.getPartition().toString()), lastRecord.getOffset() ); - } - catch (IOException e) { - throw new RuntimeException(e); - } - catch (InterruptedException e) { - throw new RuntimeException(e); - } + } } - } - closeMiniCluster(); + closeMiniCluster(); + }); } - public void hdfsReadCheck(String testConsumerTopic, int partition, long offset) throws IOException { + public void hdfsReadCheck(String testConsumerTopic, int partition, long offset) { // Check that the files were properly written to HDFS with a read test. String hdfsuri = config.getHdfsuri(); @@ -228,174 +201,176 @@ public void hdfsReadCheck(String testConsumerTopic, int partition, long offset) // Set HADOOP user System.setProperty("HADOOP_USER_NAME", "hdfs"); System.setProperty("hadoop.home.dir", "/"); - //Get the filesystem - HDFS - FileSystem fs = FileSystem.get(URI.create(hdfsuri), conf); + assertDoesNotThrow(() -> { + //Get the filesystem - HDFS + FileSystem fs = FileSystem.get(URI.create(hdfsuri), conf); - //==== Create directory if not exists - Path workingDir = fs.getWorkingDirectory(); - Path newDirectoryPath = new Path(path); - if (!fs.exists(newDirectoryPath)) { - // Create new Directory - fs.mkdirs(newDirectoryPath); - // logger.info("Path "+path+" created."); - } - - /*==== Read files - Create a path*/ - Path hdfsreadpath = new Path(newDirectoryPath + "/" + fileName); // The path should be the same that was used in writing the file to HDFS. - //Init input stream - FSDataInputStream inputStream = fs.open(hdfsreadpath); - //The data is in AVRO-format, so it can't be read as a string. - DataFileStream reader = new DataFileStream<>( - inputStream, - new SpecificDatumReader<>(SyslogRecord.class) - ); - SyslogRecord record = null; - int looper; - if (offset == 8) { - looper = 0; - } - else if (offset == 13) { - looper = 9; - } - else { - looper = 0; - Assertions.fail("The offset of the last record is not 8 or 13, which means a failed test."); - } - while (reader.hasNext()) { - record = reader.next(record); - if (LOGGER.isDebugEnabled()) { - LOGGER.debug(record.toString()); - } - // Assert records here like it is done in KafkaConsumerTest.avroReader(). - if (looper <= 0) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872090804000, \"message\": \"[WARN] 2022-04-25 07:34:50,804 com.teragrep.jla_02.Log4j Log - Log4j warn says hi!\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partition + "\", \"offset\": 0, \"origin\": \"jla-02.default\"}", - record.toString() - ); - looper++; + //==== Create directory if not exists + Path workingDir = fs.getWorkingDirectory(); + Path newDirectoryPath = new Path(path); + if (!fs.exists(newDirectoryPath)) { + // Create new Directory + fs.mkdirs(newDirectoryPath); + // logger.info("Path "+path+" created."); } - else if (looper == 1) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872090806000, \"message\": \"[ERROR] 2022-04-25 07:34:50,806 com.teragrep.jla_02.Log4j Log - Log4j error says hi!\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partition + "\", \"offset\": 1, \"origin\": \"jla-02.default\"}", - record.toString() - ); - looper++; - } - else if (looper == 2) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872090822000, \"message\": \"470647 [Thread-3] INFO com.teragrep.jla_02.Logback Daily - Logback-daily says hi.\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partition + "\", \"offset\": 2, \"origin\": \"jla-02\"}", - record.toString() - ); - looper++; - } - else if (looper == 3) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872090822000, \"message\": \"470646 [Thread-3] INFO com.teragrep.jla_02.Logback Audit - Logback-audit says hi.\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partition + "\", \"offset\": 3, \"origin\": \"jla-02\"}", - record.toString() - ); - looper++; - } - else if (looper == 4) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872090822000, \"message\": \"470647 [Thread-3] INFO com.teragrep.jla_02.Logback Metric - Logback-metric says hi.\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partition + "\", \"offset\": 4, \"origin\": \"jla-02\"}", - record.toString() - ); - looper++; - } - else if (looper == 5) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872092238000, \"message\": \"25.04.2022 07:34:52.238 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info audit says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partition + "\", \"offset\": 5, \"origin\": \"jla-02.default\"}", - record.toString() - ); - looper++; - } - else if (looper == 6) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872092239000, \"message\": \"25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info daily says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partition + "\", \"offset\": 6, \"origin\": \"jla-02.default\"}", - record.toString() - ); - looper++; - } - else if (looper == 7) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872092239000, \"message\": \"25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info metric says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partition + "\", \"offset\": 7, \"origin\": \"jla-02.default\"}", - record.toString() - ); - looper++; - } - else if (looper == 8) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872092240000, \"message\": \"25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn audit says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partition + "\", \"offset\": 8, \"origin\": \"jla-02.default\"}", - record.toString() - ); - looper++; - } - else if (looper == 9) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872092240000, \"message\": \"25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn daily says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partition + "\", \"offset\": 9, \"origin\": \"jla-02.default\"}", - record.toString() - ); - looper++; - } - else if (looper == 10) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872092241000, \"message\": \"25.04.2022 07:34:52.241 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn metric says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partition + "\", \"offset\": 10, \"origin\": \"jla-02.default\"}", - record.toString() - ); - looper++; - } - else if (looper == 11) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872092241000, \"message\": \"25.04.2022 07:34:52.241 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error audit says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partition + "\", \"offset\": 11, \"origin\": \"jla-02.default\"}", - record.toString() - ); - looper++; + + /*==== Read files + Create a path*/ + Path hdfsreadpath = new Path(newDirectoryPath + "/" + fileName); // The path should be the same that was used in writing the file to HDFS. + //Init input stream + FSDataInputStream inputStream = fs.open(hdfsreadpath); + //The data is in AVRO-format, so it can't be read as a string. + DataFileStream reader = new DataFileStream<>( + inputStream, + new SpecificDatumReader<>(SyslogRecord.class) + ); + SyslogRecord record = null; + int looper; + if (offset == 8) { + looper = 0; } - else if (looper == 12) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872092242000, \"message\": \"25.04.2022 07:34:52.242 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error daily says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partition + "\", \"offset\": 12, \"origin\": \"jla-02.default\"}", - record.toString() - ); - looper++; + else if (offset == 13) { + looper = 9; } else { - Assertions - .assertEquals( - "{\"timestamp\": 1650872092243000, \"message\": \"25.04.2022 07:34:52.243 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error metric says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partition + "\", \"offset\": 13, \"origin\": \"jla-02.default\"}", - record.toString() - ); looper = 0; + Assertions.fail("The offset of the last record is not 8 or 13, which means a failed test."); } - } - inputStream.close(); - fs.close(); + while (reader.hasNext()) { + record = reader.next(record); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug(record.toString()); + } + // Assert records here like it is done in KafkaConsumerTest.avroReader(). + if (looper <= 0) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872090804000, \"message\": \"[WARN] 2022-04-25 07:34:50,804 com.teragrep.jla_02.Log4j Log - Log4j warn says hi!\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partition + "\", \"offset\": 0, \"origin\": \"jla-02.default\"}", + record.toString() + ); + looper++; + } + else if (looper == 1) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872090806000, \"message\": \"[ERROR] 2022-04-25 07:34:50,806 com.teragrep.jla_02.Log4j Log - Log4j error says hi!\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partition + "\", \"offset\": 1, \"origin\": \"jla-02.default\"}", + record.toString() + ); + looper++; + } + else if (looper == 2) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872090822000, \"message\": \"470647 [Thread-3] INFO com.teragrep.jla_02.Logback Daily - Logback-daily says hi.\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partition + "\", \"offset\": 2, \"origin\": \"jla-02\"}", + record.toString() + ); + looper++; + } + else if (looper == 3) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872090822000, \"message\": \"470646 [Thread-3] INFO com.teragrep.jla_02.Logback Audit - Logback-audit says hi.\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partition + "\", \"offset\": 3, \"origin\": \"jla-02\"}", + record.toString() + ); + looper++; + } + else if (looper == 4) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872090822000, \"message\": \"470647 [Thread-3] INFO com.teragrep.jla_02.Logback Metric - Logback-metric says hi.\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partition + "\", \"offset\": 4, \"origin\": \"jla-02\"}", + record.toString() + ); + looper++; + } + else if (looper == 5) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872092238000, \"message\": \"25.04.2022 07:34:52.238 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info audit says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partition + "\", \"offset\": 5, \"origin\": \"jla-02.default\"}", + record.toString() + ); + looper++; + } + else if (looper == 6) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872092239000, \"message\": \"25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info daily says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partition + "\", \"offset\": 6, \"origin\": \"jla-02.default\"}", + record.toString() + ); + looper++; + } + else if (looper == 7) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872092239000, \"message\": \"25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info metric says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partition + "\", \"offset\": 7, \"origin\": \"jla-02.default\"}", + record.toString() + ); + looper++; + } + else if (looper == 8) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872092240000, \"message\": \"25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn audit says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partition + "\", \"offset\": 8, \"origin\": \"jla-02.default\"}", + record.toString() + ); + looper++; + } + else if (looper == 9) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872092240000, \"message\": \"25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn daily says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partition + "\", \"offset\": 9, \"origin\": \"jla-02.default\"}", + record.toString() + ); + looper++; + } + else if (looper == 10) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872092241000, \"message\": \"25.04.2022 07:34:52.241 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn metric says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partition + "\", \"offset\": 10, \"origin\": \"jla-02.default\"}", + record.toString() + ); + looper++; + } + else if (looper == 11) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872092241000, \"message\": \"25.04.2022 07:34:52.241 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error audit says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partition + "\", \"offset\": 11, \"origin\": \"jla-02.default\"}", + record.toString() + ); + looper++; + } + else if (looper == 12) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872092242000, \"message\": \"25.04.2022 07:34:52.242 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error daily says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partition + "\", \"offset\": 12, \"origin\": \"jla-02.default\"}", + record.toString() + ); + looper++; + } + else { + Assertions + .assertEquals( + "{\"timestamp\": 1650872092243000, \"message\": \"25.04.2022 07:34:52.243 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error metric says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partition + "\", \"offset\": 13, \"origin\": \"jla-02.default\"}", + record.toString() + ); + looper = 0; + } + } + inputStream.close(); + fs.close(); + }); } } diff --git a/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java b/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java index 85432db8..bc7bd4fb 100644 --- a/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java +++ b/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java @@ -51,6 +51,8 @@ import org.apache.avro.specific.SpecificDatumReader; import org.junit.jupiter.api.Assertions; import com.teragrep.cfe_39.avro.SyslogRecord; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -59,238 +61,222 @@ import java.nio.file.Paths; import java.io.File; -import java.io.IOException; import java.util.*; +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; + public class KafkaConsumerTest { private static final Logger LOGGER = LoggerFactory.getLogger(KafkaConsumerTest.class); // Make sure application.properties has consumer.useMockKafkaConsumer=true enabled for Kafka testing. - // @Test + @Disabled + @Test public void configTest() { // Configuration tests done, configurations working correctly with the right .jaas and .properties files. - try { + assertDoesNotThrow(() -> { Config config = new Config(); - Properties readerKafkaProperties = config.getKafkaConsumerProperties(); - // Test extracting useMockKafkaConsumer value from config. boolean useMockKafkaConsumer = Boolean .parseBoolean(readerKafkaProperties.getProperty("useMockKafkaConsumer", "false")); LOGGER.debug("useMockKafkaConsumer: " + useMockKafkaConsumer); - } - catch (IOException e) { - throw new RuntimeException(e); - } + }); } - // @Test - public void kafkaAndAvroFullTest() throws InterruptedException { - Config config = null; - try { - config = new Config(); - } - catch (IOException e) { - LOGGER.error("Can't load config: " + e); - System.exit(1); - } - catch (IllegalArgumentException e) { - LOGGER.error("Got invalid config: " + e); - System.exit(1); - } - config.setMaximumFileSize(3000); // 10 loops (140 records) are in use at the moment, and that is sized at 36,102 bytes. - HdfsDataIngestion hdfsDataIngestion = new HdfsDataIngestion(config); - hdfsDataIngestion.run(); - try { + @Disabled + @Test + public void kafkaAndAvroFullTest() { + assertDoesNotThrow(() -> { + Config config = new Config(); + config.setMaximumFileSize(3000); // 10 loops (140 records) are in use at the moment, and that is sized at 36,102 bytes. + HdfsDataIngestion hdfsDataIngestion = new HdfsDataIngestion(config); + hdfsDataIngestion.run(); int counter = avroReader(1, 2); Assertions.assertEquals(140, counter); - } - catch (IOException e) { - throw new RuntimeException(e); - } - cleanup(config, 1, 2); + cleanup(config, 1, 2); + }); } // Reads the data from a list of avro files - public int avroReader(int start, int end) throws IOException { - // Deserialize Users from disk - Config config = new Config(); - Path queueDirectory = Paths.get(config.getQueueDirectory()); - int counter = 0; - int looper = 0; - int partitionCounter = 0; - for (int j = 0; j <= 9; j++) { - for (int i = start; i <= end; i++) { - File syslogFile = new File( - queueDirectory.toAbsolutePath() + File.separator + "testConsumerTopic" + j + "." + i - ); - DatumReader userDatumReader = new SpecificDatumReader<>(SyslogRecord.class); - try (DataFileReader dataFileReader = new DataFileReader<>(syslogFile, userDatumReader)) { - SyslogRecord user = null; - while (dataFileReader.hasNext()) { - user = dataFileReader.next(user); - if (LOGGER.isDebugEnabled()) { - LOGGER.debug(syslogFile.getPath()); - LOGGER.debug(user.toString()); - } - counter++; - // All the mock data is generated from a set of 14 records. - if (looper <= 0) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872090804000, \"message\": \"[WARN] 2022-04-25 07:34:50,804 com.teragrep.jla_02.Log4j Log - Log4j warn says hi!\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 0, \"origin\": \"jla-02.default\"}", - user.toString() - ); - looper++; - } - else if (looper == 1) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872090806000, \"message\": \"[ERROR] 2022-04-25 07:34:50,806 com.teragrep.jla_02.Log4j Log - Log4j error says hi!\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 1, \"origin\": \"jla-02.default\"}", - user.toString() - ); - looper++; - } - else if (looper == 2) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872090822000, \"message\": \"470647 [Thread-3] INFO com.teragrep.jla_02.Logback Daily - Logback-daily says hi.\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter + "\", \"offset\": 2, \"origin\": \"jla-02\"}", - user.toString() - ); - looper++; - } - else if (looper == 3) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872090822000, \"message\": \"470646 [Thread-3] INFO com.teragrep.jla_02.Logback Audit - Logback-audit says hi.\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter + "\", \"offset\": 3, \"origin\": \"jla-02\"}", - user.toString() - ); - looper++; - } - else if (looper == 4) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872090822000, \"message\": \"470647 [Thread-3] INFO com.teragrep.jla_02.Logback Metric - Logback-metric says hi.\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter + "\", \"offset\": 4, \"origin\": \"jla-02\"}", - user.toString() - ); - looper++; - } - else if (looper == 5) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872092238000, \"message\": \"25.04.2022 07:34:52.238 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info audit says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 5, \"origin\": \"jla-02.default\"}", - user.toString() - ); - looper++; - } - else if (looper == 6) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872092239000, \"message\": \"25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info daily says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 6, \"origin\": \"jla-02.default\"}", - user.toString() - ); - looper++; - } - else if (looper == 7) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872092239000, \"message\": \"25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info metric says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 7, \"origin\": \"jla-02.default\"}", - user.toString() - ); - looper++; - } - else if (looper == 8) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872092240000, \"message\": \"25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn audit says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 8, \"origin\": \"jla-02.default\"}", - user.toString() - ); - looper++; - } - else if (looper == 9) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872092240000, \"message\": \"25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn daily says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 9, \"origin\": \"jla-02.default\"}", - user.toString() - ); - looper++; - } - else if (looper == 10) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872092241000, \"message\": \"25.04.2022 07:34:52.241 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn metric says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 10, \"origin\": \"jla-02.default\"}", - user.toString() - ); - looper++; - } - else if (looper == 11) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872092241000, \"message\": \"25.04.2022 07:34:52.241 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error audit says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 11, \"origin\": \"jla-02.default\"}", - user.toString() - ); - looper++; - } - else if (looper == 12) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872092242000, \"message\": \"25.04.2022 07:34:52.242 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error daily says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 12, \"origin\": \"jla-02.default\"}", - user.toString() - ); - looper++; - } - else { - Assertions - .assertEquals( - "{\"timestamp\": 1650872092243000, \"message\": \"25.04.2022 07:34:52.243 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error metric says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 13, \"origin\": \"jla-02.default\"}", - user.toString() - ); - looper = 0; - partitionCounter++; + public int avroReader(int start, int end) { + return assertDoesNotThrow(() -> { + // Deserialize Users from disk + Config config = new Config(); + Path queueDirectory = Paths.get(config.getQueueDirectory()); + int looper = 0; + int counter = 0; + int partitionCounter = 0; + for (int j = 0; j <= 9; j++) { + for (int i = start; i <= end; i++) { + File syslogFile = new File( + queueDirectory.toAbsolutePath() + File.separator + "testConsumerTopic" + j + "." + i + ); + DatumReader userDatumReader = new SpecificDatumReader<>(SyslogRecord.class); + try ( + DataFileReader dataFileReader = new DataFileReader<>( + syslogFile, + userDatumReader + ) + ) { + SyslogRecord user = null; + while (dataFileReader.hasNext()) { + user = dataFileReader.next(user); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug(syslogFile.getPath()); + LOGGER.debug(user.toString()); + } + counter++; + // All the mock data is generated from a set of 14 records. + if (looper <= 0) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872090804000, \"message\": \"[WARN] 2022-04-25 07:34:50,804 com.teragrep.jla_02.Log4j Log - Log4j warn says hi!\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 0, \"origin\": \"jla-02.default\"}", + user.toString() + ); + looper++; + } + else if (looper == 1) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872090806000, \"message\": \"[ERROR] 2022-04-25 07:34:50,806 com.teragrep.jla_02.Log4j Log - Log4j error says hi!\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 1, \"origin\": \"jla-02.default\"}", + user.toString() + ); + looper++; + } + else if (looper == 2) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872090822000, \"message\": \"470647 [Thread-3] INFO com.teragrep.jla_02.Logback Daily - Logback-daily says hi.\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 2, \"origin\": \"jla-02\"}", + user.toString() + ); + looper++; + } + else if (looper == 3) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872090822000, \"message\": \"470646 [Thread-3] INFO com.teragrep.jla_02.Logback Audit - Logback-audit says hi.\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 3, \"origin\": \"jla-02\"}", + user.toString() + ); + looper++; + } + else if (looper == 4) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872090822000, \"message\": \"470647 [Thread-3] INFO com.teragrep.jla_02.Logback Metric - Logback-metric says hi.\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 4, \"origin\": \"jla-02\"}", + user.toString() + ); + looper++; + } + else if (looper == 5) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872092238000, \"message\": \"25.04.2022 07:34:52.238 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info audit says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 5, \"origin\": \"jla-02.default\"}", + user.toString() + ); + looper++; + } + else if (looper == 6) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872092239000, \"message\": \"25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info daily says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 6, \"origin\": \"jla-02.default\"}", + user.toString() + ); + looper++; + } + else if (looper == 7) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872092239000, \"message\": \"25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info metric says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 7, \"origin\": \"jla-02.default\"}", + user.toString() + ); + looper++; + } + else if (looper == 8) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872092240000, \"message\": \"25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn audit says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 8, \"origin\": \"jla-02.default\"}", + user.toString() + ); + looper++; + } + else if (looper == 9) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872092240000, \"message\": \"25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn daily says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 9, \"origin\": \"jla-02.default\"}", + user.toString() + ); + looper++; + } + else if (looper == 10) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872092241000, \"message\": \"25.04.2022 07:34:52.241 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn metric says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 10, \"origin\": \"jla-02.default\"}", + user.toString() + ); + looper++; + } + else if (looper == 11) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872092241000, \"message\": \"25.04.2022 07:34:52.241 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error audit says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 11, \"origin\": \"jla-02.default\"}", + user.toString() + ); + looper++; + } + else if (looper == 12) { + Assertions + .assertEquals( + "{\"timestamp\": 1650872092242000, \"message\": \"25.04.2022 07:34:52.242 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error daily says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 12, \"origin\": \"jla-02.default\"}", + user.toString() + ); + looper++; + } + else { + Assertions + .assertEquals( + "{\"timestamp\": 1650872092243000, \"message\": \"25.04.2022 07:34:52.243 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error metric says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 13, \"origin\": \"jla-02.default\"}", + user.toString() + ); + looper = 0; + partitionCounter++; + } } + } } } - } - LOGGER.debug("Total number of records: " + counter); - return counter; - } - - // @Test - public void debugger() { - try { - int counter = avroReader(1, 2); - Assertions.assertEquals(140, counter); - } - catch (IOException e) { - throw new RuntimeException(e); - } + LOGGER.debug("Total number of records: " + counter); + return counter; + }); } // Deletes the avro-files that were created during testing. @@ -301,12 +287,9 @@ public void cleanup(Config config, int start, int end) { File syslogFile = new File( queueDirectory.toAbsolutePath() + File.separator + "testConsumerTopic" + j + "." + i ); - try { - boolean result = Files.deleteIfExists(syslogFile.toPath()); //surround it in try catch block - } - catch (IOException e) { - throw new RuntimeException(e); - } + assertDoesNotThrow(() -> { + boolean result = Files.deleteIfExists(syslogFile.toPath()); + }); } } } From 18860a44d66936eb459d8bbe95429e7aedf8e437 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Fri, 7 Jun 2024 11:48:53 +0300 Subject: [PATCH 086/146] Added PruningTest.java (WIP). Minor tweak to HDFSPrune.prune() to return number of successfully pruned files. --- .../cfe_39/consumers/kafka/HDFSPrune.java | 5 +- .../java/com/teragrep/cfe_39/PruningTest.java | 197 ++++++++++++++++++ 2 files changed, 201 insertions(+), 1 deletion(-) create mode 100644 src/test/java/com/teragrep/cfe_39/PruningTest.java diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSPrune.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSPrune.java index 3853093e..38c1988a 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSPrune.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSPrune.java @@ -150,7 +150,8 @@ public HDFSPrune(Config config, String topicName) throws IOException { cutOffEpoch = System.currentTimeMillis() - pruneOffset; // pruneOffset is parametrized in Config.java. Default value is 2 days in milliseconds. } - public void prune() throws IOException { + public int prune() throws IOException { + int deleted = 0; // Fetch the filestatuses of HDFS files. FileStatus[] fileStatuses = fs.listStatus(new Path(newDirectoryPath + "/")); if (fileStatuses.length > 0) { @@ -159,11 +160,13 @@ public void prune() throws IOException { if (a.getModificationTime() < cutOffEpoch) { boolean delete = fs.delete(a.getPath(), true); LOGGER.info("Deleted file <{}>", a.getPath()); + deleted++; } } } else { LOGGER.info("No files found in directory <{}>", new Path(newDirectoryPath + "/")); } + return deleted; } } diff --git a/src/test/java/com/teragrep/cfe_39/PruningTest.java b/src/test/java/com/teragrep/cfe_39/PruningTest.java new file mode 100644 index 00000000..86ca0236 --- /dev/null +++ b/src/test/java/com/teragrep/cfe_39/PruningTest.java @@ -0,0 +1,197 @@ +/* + * HDFS Data Ingestion for PTH_06 use CFE-39 + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. + */ +package com.teragrep.cfe_39; + +import com.teragrep.cfe_39.consumers.kafka.HDFSPrune; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FileUtil; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.net.URI; +import java.nio.file.Files; +import java.util.Objects; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; + +public class PruningTest { + + private static final Logger LOGGER = LoggerFactory.getLogger(CombinedFullTest.class); + private static MiniDFSCluster hdfsCluster; + private static File baseDir; + private static Config config; + private FileSystem fs; + + // Start minicluster and initialize config. + @BeforeEach + public void startMiniCluster() { + assertDoesNotThrow(() -> { + config = new Config(); + // Create a HDFS miniCluster + baseDir = Files.createTempDirectory("test_hdfs").toFile().getAbsoluteFile(); + Configuration conf = new Configuration(); + conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, baseDir.getAbsolutePath()); + MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf); + hdfsCluster = builder.build(); + String hdfsURI = "hdfs://localhost:" + hdfsCluster.getNameNodePort() + "/"; + config.setHdfsuri(hdfsURI); + DistributedFileSystem fileSystem = hdfsCluster.getFileSystem(); + + // ====== Init HDFS File System Object + Configuration fsConf = new Configuration(); + // Set FileSystem URI + fsConf.set("fs.defaultFS", hdfsURI); + // Because of Maven + fsConf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); + fsConf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); + // Set HADOOP user + System.setProperty("HADOOP_USER_NAME", "hdfs"); + System.setProperty("hadoop.home.dir", "/"); + String path = config.getHdfsPath() + "/" + "testConsumerTopic"; // "hdfs:///opt/teragrep/cfe_39/srv/testConsumerTopic" + fs = FileSystem.get(URI.create(hdfsURI), fsConf); + }); + + } + + // Teardown the minicluster + @AfterEach + public void teardownMiniCluster() { + assertDoesNotThrow(() -> { + fs.close(); + }); + hdfsCluster.shutdown(); + FileUtil.fullyDelete(baseDir); + } + + /* TODO: Create more extensive tests based on happyTest: + - HDFS has 1 file already inside it that is timestamped as young enough that it is not pruned. + - HDFS has 2 files already inside it that are timestamped as young enough that they are not pruned. happyTest() + - HDFS has 1 file already inside it that is timestamped as old enough for pruning to trigger. + - HDFS has 2 files already inside it that are timestamped as old enough for pruning to trigger. + - HDFS has 2 files already inside it where one file is old enough to be pruned and the other one is not.*/ + + @Test + public void happyTest() { + // Test for not triggering pruning for files in the topic. + Assertions.assertTrue(config.getPruneOffset() >= 300000L); // Fails the test if the config is not correct, too low pruning offset can prune the files if the test is lagging. + insertMockFiles(-1, -1); // Insert 2 mock files normally with young timestamps so pruning should not trigger on them. + + assertDoesNotThrow(() -> { + HDFSPrune hdfsPrune = new HDFSPrune(config, "testConsumerTopic"); + int deleted = hdfsPrune.prune(); + Assertions.assertEquals(0, deleted); + // Also check with HDFS access if expected files still exist. + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.13"))); + }); + } + + // Inserts pre-made avro-files to HDFS, which are normally generated during data ingestion from mock kafka consumer. + private void insertMockFiles(long a, long b) { + String path = config.getHdfsPath() + "/" + "testConsumerTopic"; // "hdfs:///opt/teragrep/cfe_39/srv/testConsumerTopic" + //Get the filesystem - HDFS + assertDoesNotThrow(() -> { + + //==== Create directory if not exists + Path workingDir = fs.getWorkingDirectory(); + // Sets the directory where the data should be stored, if the directory doesn't exist then it's created. + Path newDirectoryPath = new Path(path); + if (!fs.exists(newDirectoryPath)) { + // Create new Directory + fs.mkdirs(newDirectoryPath); + LOGGER.debug("Path {} created.", path); + } + + String dir = System.getProperty("user.dir") + "/src/test/java/com/teragrep/cfe_39/mockHdfsFiles"; + Set listOfFiles = Stream + .of(Objects.requireNonNull(new File(dir).listFiles())) + .filter(file -> !file.isDirectory()) + .map(File::getName) + .collect(Collectors.toSet()); + // Loop through all the avro files + for (String fileName : listOfFiles) { + String pathname = dir + "/" + fileName; + File avroFile = new File(pathname); + //==== Write file + LOGGER.debug("Begin Write file into hdfs"); + //Create a path + Path hdfswritepath = new Path(newDirectoryPath + "/" + avroFile.getName()); // filename should be set according to the requirements: 0.12345 where 0 is Kafka partition and 12345 is Kafka offset. + if (fs.exists(hdfswritepath)) { + Assertions.fail("File " + avroFile.getName() + " already exists"); + } + Path readPath = new Path(avroFile.getPath()); + // Add conditions if file filtering is required for tests. + fs.copyFromLocalFile(readPath, hdfswritepath); + // Set a/b to something like 157784760000 to trigger pruning. + if (Objects.equals(hdfswritepath.toString(), "hdfs:/opt/teragrep/cfe_39/srv/testConsumerTopic/0.9")) { + fs.setTimes(hdfswritepath, a, -1); + } + else if ( + Objects.equals(hdfswritepath.toString(), "hdfs:/opt/teragrep/cfe_39/srv/testConsumerTopic/0.13") + ) { + fs.setTimes(hdfswritepath, b, -1); + } + LOGGER.debug("End Write file into hdfs"); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug("\nFile committed to HDFS, file writepath should be: {}\n", hdfswritepath.toString()); + } + } + }); + } + +} From b015ca735263e79b17bbae76784b3cb299c3516d Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Fri, 7 Jun 2024 12:16:40 +0300 Subject: [PATCH 087/146] Spotless --- .../com/teragrep/cfe_39/CombinedFullTest.java | 25 +++++++++++-------- 1 file changed, 14 insertions(+), 11 deletions(-) diff --git a/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java b/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java index 466c179c..bb41c436 100644 --- a/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java +++ b/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java @@ -160,16 +160,16 @@ public void hdfsPruneTest() { long count = Arrays.stream(fileStatuses).count(); Assertions.assertTrue(count > 0); - for (FileStatus a : fileStatuses) { - // Delete old files - if (a.getModificationTime() < System.currentTimeMillis()) { - boolean deleted = fs.delete(a.getPath(), true); - Assertions.assertTrue(deleted); - LOGGER.info("Deleted file {}", a.getPath()); - LOGGER.debug("Timestamp was {}", a.getModificationTime()); // example: 1717582386630 milliseconds which translates to Wednesday, June 5, 2024 1:13:06.630 PM GMT+03:00 DST - } + for (FileStatus a : fileStatuses) { + // Delete old files + if (a.getModificationTime() < System.currentTimeMillis()) { + boolean deleted = fs.delete(a.getPath(), true); + Assertions.assertTrue(deleted); + LOGGER.info("Deleted file {}", a.getPath()); + LOGGER.debug("Timestamp was {}", a.getModificationTime()); // example: 1717582386630 milliseconds which translates to Wednesday, June 5, 2024 1:13:06.630 PM GMT+03:00 DST } - LOGGER.info("All files were pruned properly."); + } + LOGGER.info("All files were pruned properly."); }); } @@ -199,7 +199,7 @@ public void hdfsReadCheck() { /* This is the HDFS write path for the files: Path hdfswritepath = new Path(newDirectoryPath + "/" + fileName); where newDirectoryPath is config.getHdfsPath() + "/" + lastObject.topic; and filename is lastObject.partition+"."+lastObject.offset; - + Create the list of files to read from HDFS. Test setup is created so each of the 0-9 partitions will have 1 file with offset of 13.*/ List filenameList = new ArrayList<>(); for (int i = 0; i <= 9; i++) { @@ -457,7 +457,10 @@ private void insertMockFiles(long a, long b) { // Set a/b to something like 157784760000 to trigger pruning. if (Objects.equals(hdfswritepath.toString(), "hdfs:/opt/teragrep/cfe_39/srv/testConsumerTopic/0.9")) { fs.setTimes(hdfswritepath, a, -1); - } else if (Objects.equals(hdfswritepath.toString(), "hdfs:/opt/teragrep/cfe_39/srv/testConsumerTopic/0.13")) { + } + else if ( + Objects.equals(hdfswritepath.toString(), "hdfs:/opt/teragrep/cfe_39/srv/testConsumerTopic/0.13") + ) { fs.setTimes(hdfswritepath, b, -1); } LOGGER.debug("End Write file into hdfs"); From 61c54a1ae34fa8ab4dec0fa0e69d9eb1bb506d1c Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Fri, 7 Jun 2024 15:55:09 +0300 Subject: [PATCH 088/146] Expanded and improved tests in PruningTest.java --- .../java/com/teragrep/cfe_39/PruningTest.java | 189 +++++++++++++++--- 1 file changed, 161 insertions(+), 28 deletions(-) diff --git a/src/test/java/com/teragrep/cfe_39/PruningTest.java b/src/test/java/com/teragrep/cfe_39/PruningTest.java index 86ca0236..49c0472d 100644 --- a/src/test/java/com/teragrep/cfe_39/PruningTest.java +++ b/src/test/java/com/teragrep/cfe_39/PruningTest.java @@ -71,7 +71,7 @@ public class PruningTest { - private static final Logger LOGGER = LoggerFactory.getLogger(CombinedFullTest.class); + private static final Logger LOGGER = LoggerFactory.getLogger(PruningTest.class); private static MiniDFSCluster hdfsCluster; private static File baseDir; private static Config config; @@ -102,7 +102,6 @@ public void startMiniCluster() { // Set HADOOP user System.setProperty("HADOOP_USER_NAME", "hdfs"); System.setProperty("hadoop.home.dir", "/"); - String path = config.getHdfsPath() + "/" + "testConsumerTopic"; // "hdfs:///opt/teragrep/cfe_39/srv/testConsumerTopic" fs = FileSystem.get(URI.create(hdfsURI), fsConf); }); @@ -111,44 +110,146 @@ public void startMiniCluster() { // Teardown the minicluster @AfterEach public void teardownMiniCluster() { - assertDoesNotThrow(() -> { - fs.close(); - }); + assertDoesNotThrow(fs::close); hdfsCluster.shutdown(); FileUtil.fullyDelete(baseDir); } - /* TODO: Create more extensive tests based on happyTest: - - HDFS has 1 file already inside it that is timestamped as young enough that it is not pruned. - - HDFS has 2 files already inside it that are timestamped as young enough that they are not pruned. happyTest() - - HDFS has 1 file already inside it that is timestamped as old enough for pruning to trigger. - - HDFS has 2 files already inside it that are timestamped as old enough for pruning to trigger. - - HDFS has 2 files already inside it where one file is old enough to be pruned and the other one is not.*/ - @Test - public void happyTest() { - // Test for not triggering pruning for files in the topic. + public void twoNewFiles() { + // Test for not triggering pruning for 2 files in the topic. Assertions.assertTrue(config.getPruneOffset() >= 300000L); // Fails the test if the config is not correct, too low pruning offset can prune the files if the test is lagging. - insertMockFiles(-1, -1); // Insert 2 mock files normally with young timestamps so pruning should not trigger on them. + insertMockFiles(-1, -1); // Insert 2 mock files with new timestamps so pruning should not trigger on them. assertDoesNotThrow(() -> { + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.13"))); HDFSPrune hdfsPrune = new HDFSPrune(config, "testConsumerTopic"); int deleted = hdfsPrune.prune(); Assertions.assertEquals(0, deleted); // Also check with HDFS access if expected files still exist. + Assertions + .assertEquals(2, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.13"))); + }); + } + + @Test + public void noFiles() { + assertDoesNotThrow(() -> { + Assertions.assertFalse(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic"))); + HDFSPrune hdfsPrune = new HDFSPrune(config, "testConsumerTopic"); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic"))); + Assertions + .assertEquals(0, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); + int deleted = hdfsPrune.prune(); + Assertions.assertEquals(0, deleted); + Assertions + .assertEquals(0, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); + }); + } + + @Test + public void twoOldFiles() { + // Test for triggering pruning for 2 files in the topic. + Assertions.assertTrue(config.getPruneOffset() >= 300000L); // Fails the test if the config is not correct, too low pruning offset can prune the files if the test is lagging. + Assertions.assertTrue(System.currentTimeMillis() - config.getPruneOffset() > 157784760000L); + insertMockFiles(157784760000L, 157784760000L); // Insert 2 mock files with old timestamps so pruning should trigger on them. + + assertDoesNotThrow(() -> { + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic"))); + Assertions + .assertEquals(2, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.13"))); + HDFSPrune hdfsPrune = new HDFSPrune(config, "testConsumerTopic"); + int deleted = hdfsPrune.prune(); + Assertions.assertEquals(2, deleted); + // Also check with HDFS access if expected files still exist. + Assertions + .assertEquals(0, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); + Assertions.assertFalse(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); + Assertions + .assertFalse(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.13"))); + }); + } + + @Test + public void oneOldOneNewFile() { + // Test for triggering pruning for 1 out of 2 files in the topic. + Assertions.assertTrue(config.getPruneOffset() >= 300000L); // Fails the test if the config is not correct, too low pruning offset can prune the files if the test is lagging. + Assertions.assertTrue(System.currentTimeMillis() - config.getPruneOffset() > 157784760000L); + insertMockFiles(157784760000L, -1); // Insert 2 mock files, one with old timestamp and one with new timestamp. + + assertDoesNotThrow(() -> { + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic"))); + Assertions + .assertEquals(2, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.13"))); + HDFSPrune hdfsPrune = new HDFSPrune(config, "testConsumerTopic"); + int deleted = hdfsPrune.prune(); + Assertions.assertEquals(1, deleted); + // Also check with HDFS access if expected files still exist. + Assertions + .assertEquals(1, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); + Assertions.assertFalse(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.13"))); + }); + } + + @Test + public void oneOldFile() { + // Test for triggering pruning for a single file. + Assertions.assertTrue(config.getPruneOffset() >= 300000L); // Fails the test if the config is not correct, too low pruning offset can prune the files if the test is lagging. + Assertions.assertTrue(System.currentTimeMillis() - config.getPruneOffset() > 157784760000L); + insertMockFile(157784760000L); // Insert 1 old mock file. + + assertDoesNotThrow(() -> { + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic"))); + Assertions + .assertEquals(1, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); + HDFSPrune hdfsPrune = new HDFSPrune(config, "testConsumerTopic"); + int deleted = hdfsPrune.prune(); + Assertions.assertEquals(1, deleted); + // Also check with HDFS access if expected files still exist. + Assertions + .assertEquals(0, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); + Assertions.assertFalse(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); + }); + } + + @Test + public void oneNewFile() { + // Test for not triggering pruning for a single file. + Assertions.assertTrue(config.getPruneOffset() >= 300000L); // Fails the test if the config is not correct, too low pruning offset can prune the files if the test is lagging. + Assertions.assertTrue(System.currentTimeMillis() - config.getPruneOffset() > 157784760000L); + insertMockFile(-1); // Insert 1 new mock file. + + assertDoesNotThrow(() -> { + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic"))); + Assertions + .assertEquals(1, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); + HDFSPrune hdfsPrune = new HDFSPrune(config, "testConsumerTopic"); + int deleted = hdfsPrune.prune(); + Assertions.assertEquals(0, deleted); + // Also check with HDFS access if expected files still exist. + Assertions + .assertEquals(1, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); }); } // Inserts pre-made avro-files to HDFS, which are normally generated during data ingestion from mock kafka consumer. - private void insertMockFiles(long a, long b) { + private void insertMockFiles(long fileTimestampA, long fileTimestampB) { String path = config.getHdfsPath() + "/" + "testConsumerTopic"; // "hdfs:///opt/teragrep/cfe_39/srv/testConsumerTopic" //Get the filesystem - HDFS assertDoesNotThrow(() -> { - //==== Create directory if not exists - Path workingDir = fs.getWorkingDirectory(); // Sets the directory where the data should be stored, if the directory doesn't exist then it's created. Path newDirectoryPath = new Path(path); if (!fs.exists(newDirectoryPath)) { @@ -171,27 +272,59 @@ private void insertMockFiles(long a, long b) { LOGGER.debug("Begin Write file into hdfs"); //Create a path Path hdfswritepath = new Path(newDirectoryPath + "/" + avroFile.getName()); // filename should be set according to the requirements: 0.12345 where 0 is Kafka partition and 12345 is Kafka offset. - if (fs.exists(hdfswritepath)) { - Assertions.fail("File " + avroFile.getName() + " already exists"); - } + Assertions.assertFalse(fs.exists(hdfswritepath)); Path readPath = new Path(avroFile.getPath()); - // Add conditions if file filtering is required for tests. fs.copyFromLocalFile(readPath, hdfswritepath); - // Set a/b to something like 157784760000 to trigger pruning. + // Set fileTimestampA/fileTimestampB to something like 157784760000 to trigger pruning, -1 to not alter the timestamp. if (Objects.equals(hdfswritepath.toString(), "hdfs:/opt/teragrep/cfe_39/srv/testConsumerTopic/0.9")) { - fs.setTimes(hdfswritepath, a, -1); + fs.setTimes(hdfswritepath, fileTimestampA, -1); } else if ( Objects.equals(hdfswritepath.toString(), "hdfs:/opt/teragrep/cfe_39/srv/testConsumerTopic/0.13") ) { - fs.setTimes(hdfswritepath, b, -1); + fs.setTimes(hdfswritepath, fileTimestampB, -1); } LOGGER.debug("End Write file into hdfs"); - if (LOGGER.isDebugEnabled()) { - LOGGER.debug("\nFile committed to HDFS, file writepath should be: {}\n", hdfswritepath.toString()); - } + LOGGER.debug("\nFile committed to HDFS, file writepath should be: {}\n", hdfswritepath); } }); } + // Inserts a single pre-made avro-file to HDFS, which is normally generated during data ingestion from mock kafka consumer. + private void insertMockFile(long fileTimestamp) { + String path = config.getHdfsPath() + "/" + "testConsumerTopic"; // "hdfs:///opt/teragrep/cfe_39/srv/testConsumerTopic" + assertDoesNotThrow(() -> { + + // Sets the directory where the data should be stored, if the directory doesn't exist then it's created. + Path newDirectoryPath = new Path(path); + if (!fs.exists(newDirectoryPath)) { + // Create new Directory + fs.mkdirs(newDirectoryPath); + LOGGER.debug("Path {} created.", path); + } + + String dir = System.getProperty("user.dir") + "/src/test/java/com/teragrep/cfe_39/mockHdfsFiles"; + Set listOfFiles = Stream + .of(Objects.requireNonNull(new File(dir).listFiles())) + .filter(file -> !file.isDirectory()) + .map(File::getName) + .collect(Collectors.toSet()); + String fileName = "0.9"; + Assertions.assertTrue(listOfFiles.contains(fileName)); + String pathname = dir + "/" + fileName; + File avroFile = new File(pathname); + //==== Write file + LOGGER.debug("Begin Write file into hdfs"); + //Create a path + Path hdfswritepath = new Path(newDirectoryPath + "/" + avroFile.getName()); // filename should be set according to the requirements: 0.12345 where 0 is Kafka partition and 12345 is Kafka offset. + Assertions.assertFalse(fs.exists(hdfswritepath)); + Path readPath = new Path(avroFile.getPath()); + fs.copyFromLocalFile(readPath, hdfswritepath); + // Set fileTimestamp to something like 157784760000 to trigger pruning, -1 to not alter the timestamp. + fs.setTimes(hdfswritepath, fileTimestamp, -1); + LOGGER.debug("End Write file into hdfs"); + LOGGER.debug("\nFile committed to HDFS, file writepath should be: {}\n", hdfswritepath); + }); + } + } From 9c3b842a6d422bb47b619d8dd91e670a1f6319f0 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Tue, 11 Jun 2024 13:10:17 +0300 Subject: [PATCH 089/146] Added ConfigTest.java. --- .../java/com/teragrep/cfe_39/ConfigTest.java | 71 +++++++++++++++++++ 1 file changed, 71 insertions(+) create mode 100644 src/test/java/com/teragrep/cfe_39/ConfigTest.java diff --git a/src/test/java/com/teragrep/cfe_39/ConfigTest.java b/src/test/java/com/teragrep/cfe_39/ConfigTest.java new file mode 100644 index 00000000..8a8a0ea5 --- /dev/null +++ b/src/test/java/com/teragrep/cfe_39/ConfigTest.java @@ -0,0 +1,71 @@ +/* + * HDFS Data Ingestion for PTH_06 use CFE-39 + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. + */ +package com.teragrep.cfe_39; + +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Properties; + +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; + +public class ConfigTest { + + private static final Logger LOGGER = LoggerFactory.getLogger(ConfigTest.class); + + @Test + public void configTest() { + assertDoesNotThrow(() -> { + Config config = new Config(); + Properties readerKafkaProperties = config.getKafkaConsumerProperties(); + // Test extracting useMockKafkaConsumer value from config. + boolean useMockKafkaConsumer = Boolean + .parseBoolean(readerKafkaProperties.getProperty("useMockKafkaConsumer", "false")); + LOGGER.debug("useMockKafkaConsumer: " + useMockKafkaConsumer); + }); + } +} From 6dd01c4e2ac4bdfe4d363af65f886677afe7391d Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Tue, 11 Jun 2024 16:17:09 +0300 Subject: [PATCH 090/146] Refactored KafkaConsumerTest.java. Added condition to ReadCoordinator.java to allow using a single consumer instead of two consumer group for testing. --- .../consumers/kafka/ReadCoordinator.java | 5 +- .../teragrep/cfe_39/KafkaConsumerTest.java | 688 ++++++++++++------ 2 files changed, 472 insertions(+), 221 deletions(-) diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinator.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinator.java index c5c45694..e28ca98b 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinator.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinator.java @@ -89,9 +89,12 @@ private KafkaReader createKafkaReader( if (Objects.equals(name, "testConsumerTopic1")) { kafkaConsumer = MockKafkaConsumerFactory.getConsumer(1); // creates a Kafka MockConsumer that has the odd numbered partitions assigned to it. } - else { + else if (Objects.equals(name, "testConsumerTopic2")) { kafkaConsumer = MockKafkaConsumerFactory.getConsumer(2); // creates a Kafka MockConsumer that has the even numbered partitions assigned to it. } + else { + kafkaConsumer = MockKafkaConsumerFactory.getConsumer(0); // Creates a single Kafka MockConsumer that has all the partitions assigned to it. + } } else { // Mock kafka consumer is disabled, subscribe method should handle assigning the partitions automatically to the consumer based on group id parameters of readerKafkaProperties. kafkaConsumer = new KafkaConsumer<>( diff --git a/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java b/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java index bc7bd4fb..e5a4155b 100644 --- a/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java +++ b/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java @@ -45,252 +45,500 @@ */ package com.teragrep.cfe_39; -import com.teragrep.cfe_39.consumers.kafka.HdfsDataIngestion; -import org.apache.avro.file.DataFileReader; -import org.apache.avro.io.DatumReader; -import org.apache.avro.specific.SpecificDatumReader; +import com.teragrep.cfe_39.consumers.kafka.ReadCoordinator; +import com.teragrep.cfe_39.consumers.kafka.RecordOffset; +import com.teragrep.rlo_06.RFC5424Frame; +import org.apache.kafka.common.TopicPartition; import org.junit.jupiter.api.Assertions; -import com.teragrep.cfe_39.avro.SyslogRecord; -import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.nio.file.Files; -import java.nio.file.Path; -import java.nio.file.Paths; - -import java.io.File; +import java.io.ByteArrayInputStream; import java.util.*; +import java.util.function.Consumer; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; public class KafkaConsumerTest { private static final Logger LOGGER = LoggerFactory.getLogger(KafkaConsumerTest.class); - // Make sure application.properties has consumer.useMockKafkaConsumer=true enabled for Kafka testing. - @Disabled @Test - public void configTest() { - // Configuration tests done, configurations working correctly with the right .jaas and .properties files. + public void readCoordinatorTest2Threads() { assertDoesNotThrow(() -> { Config config = new Config(); - Properties readerKafkaProperties = config.getKafkaConsumerProperties(); - // Test extracting useMockKafkaConsumer value from config. - boolean useMockKafkaConsumer = Boolean - .parseBoolean(readerKafkaProperties.getProperty("useMockKafkaConsumer", "false")); - LOGGER.debug("useMockKafkaConsumer: " + useMockKafkaConsumer); + Map hdfsStartOffsets = new HashMap<>(); + ArrayList> messages = new ArrayList<>(); + Consumer> output = message -> messages.add(message); + + ReadCoordinator readCoordinator = new ReadCoordinator( + "testConsumerTopic", + config.getKafkaConsumerProperties(), + output, + hdfsStartOffsets + ); + Thread readThread = new Thread(null, readCoordinator, "testConsumerTopic1"); // Starts the thread with readCoordinator that creates the consumer and subscribes to the topic. + readThread.start(); // Starts the thread, in other words proceeds to call run() function of ReadCoordinator. + + Thread.sleep(1000); + + ReadCoordinator readCoordinator2 = new ReadCoordinator( + "testConsumerTopic", + config.getKafkaConsumerProperties(), + output, + hdfsStartOffsets + ); + Thread readThread2 = new Thread(null, readCoordinator2, "testConsumerTopic2"); // Starts the thread with readCoordinator that creates the consumer and subscribes to the topic. + readThread2.start(); // Starts the thread, in other words proceeds to call run() function of ReadCoordinator. + + Thread.sleep(10000); + Assertions.assertEquals(2, messages.size()); + Assertions.assertEquals(140, messages.get(0).size() + messages.get(1).size()); // Assert that expected amount of records has been consumed by the consumer group. + Assertions.assertEquals(70, messages.get(0).size()); + Assertions.assertEquals(70, messages.get(1).size()); + + // Assert that all the record contents are correct, every topic partition has identical set of offset-message pairings. + List messageList = new ArrayList(); + messageList.add("[WARN] 2022-04-25 07:34:50,804 com.teragrep.jla_02.Log4j Log - Log4j warn says hi!"); + messageList.add("[ERROR] 2022-04-25 07:34:50,806 com.teragrep.jla_02.Log4j Log - Log4j error says hi!"); + messageList.add("470647 [Thread-3] INFO com.teragrep.jla_02.Logback Daily - Logback-daily says hi."); + messageList.add("470646 [Thread-3] INFO com.teragrep.jla_02.Logback Audit - Logback-audit says hi."); + messageList.add("470647 [Thread-3] INFO com.teragrep.jla_02.Logback Metric - Logback-metric says hi."); + messageList + .add( + "25.04.2022 07:34:52.238 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info audit says hi!]" + ); + messageList + .add( + "25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info daily says hi!]" + ); + messageList + .add( + "25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info metric says hi!]" + ); + messageList + .add( + "25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn audit says hi!]" + ); + messageList + .add( + "25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn daily says hi!]" + ); + messageList + .add( + "25.04.2022 07:34:52.241 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn metric says hi!]" + ); + messageList + .add( + "25.04.2022 07:34:52.241 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error audit says hi!]" + ); + messageList + .add( + "25.04.2022 07:34:52.242 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error daily says hi!]" + ); + messageList + .add( + "25.04.2022 07:34:52.243 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error metric says hi!]" + ); + + RFC5424Frame rfc5424Frame = new RFC5424Frame(false); + + RecordOffset recordOffset; + + Iterator iterator = messageList.iterator(); + int counter = 0; + for (int i = 0; i <= 13; i++) { + recordOffset = messages.get(0).get(counter); + Assertions + .assertEquals( + "{\"topic\":\"testConsumerTopic\", \"partition\":\"7\", \"offset\":\"" + i + "\"}", + recordOffset.offsetToJSON() + ); + rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); + Assertions.assertTrue(rfc5424Frame.next()); + Assertions.assertTrue(iterator.hasNext()); + Assertions.assertEquals(iterator.next(), rfc5424Frame.msg.toString()); + Assertions.assertFalse(rfc5424Frame.next()); + counter++; + } + + iterator = messageList.iterator(); + for (int i = 0; i <= 13; i++) { + recordOffset = messages.get(0).get(counter); + Assertions + .assertEquals( + "{\"topic\":\"testConsumerTopic\", \"partition\":\"5\", \"offset\":\"" + i + "\"}", + recordOffset.offsetToJSON() + ); + rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); + Assertions.assertTrue(rfc5424Frame.next()); + Assertions.assertTrue(iterator.hasNext()); + Assertions.assertEquals(iterator.next(), rfc5424Frame.msg.toString()); + Assertions.assertFalse(rfc5424Frame.next()); + counter++; + } + + iterator = messageList.iterator(); + for (int i = 0; i <= 13; i++) { + recordOffset = messages.get(0).get(counter); + Assertions + .assertEquals( + "{\"topic\":\"testConsumerTopic\", \"partition\":\"3\", \"offset\":\"" + i + "\"}", + recordOffset.offsetToJSON() + ); + rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); + Assertions.assertTrue(rfc5424Frame.next()); + Assertions.assertTrue(iterator.hasNext()); + Assertions.assertEquals(iterator.next(), rfc5424Frame.msg.toString()); + Assertions.assertFalse(rfc5424Frame.next()); + counter++; + } + + iterator = messageList.iterator(); + for (int i = 0; i <= 13; i++) { + recordOffset = messages.get(0).get(counter); + Assertions + .assertEquals( + "{\"topic\":\"testConsumerTopic\", \"partition\":\"1\", \"offset\":\"" + i + "\"}", + recordOffset.offsetToJSON() + ); + rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); + Assertions.assertTrue(rfc5424Frame.next()); + Assertions.assertTrue(iterator.hasNext()); + Assertions.assertEquals(iterator.next(), rfc5424Frame.msg.toString()); + Assertions.assertFalse(rfc5424Frame.next()); + counter++; + } + + iterator = messageList.iterator(); + for (int i = 0; i <= 13; i++) { + recordOffset = messages.get(0).get(counter); + Assertions + .assertEquals( + "{\"topic\":\"testConsumerTopic\", \"partition\":\"9\", \"offset\":\"" + i + "\"}", + recordOffset.offsetToJSON() + ); + rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); + Assertions.assertTrue(rfc5424Frame.next()); + Assertions.assertTrue(iterator.hasNext()); + Assertions.assertEquals(iterator.next(), rfc5424Frame.msg.toString()); + Assertions.assertFalse(rfc5424Frame.next()); + counter++; + } + + Assertions.assertEquals(70, counter); + + counter = 0; + iterator = messageList.iterator(); + for (int i = 0; i <= 13; i++) { + recordOffset = messages.get(1).get(counter); + Assertions + .assertEquals( + "{\"topic\":\"testConsumerTopic\", \"partition\":\"8\", \"offset\":\"" + i + "\"}", + recordOffset.offsetToJSON() + ); + rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); + Assertions.assertTrue(rfc5424Frame.next()); + Assertions.assertTrue(iterator.hasNext()); + Assertions.assertEquals(iterator.next(), rfc5424Frame.msg.toString()); + Assertions.assertFalse(rfc5424Frame.next()); + counter++; + } + + iterator = messageList.iterator(); + for (int i = 0; i <= 13; i++) { + recordOffset = messages.get(1).get(counter); + Assertions + .assertEquals( + "{\"topic\":\"testConsumerTopic\", \"partition\":\"6\", \"offset\":\"" + i + "\"}", + recordOffset.offsetToJSON() + ); + rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); + Assertions.assertTrue(rfc5424Frame.next()); + Assertions.assertTrue(iterator.hasNext()); + Assertions.assertEquals(iterator.next(), rfc5424Frame.msg.toString()); + Assertions.assertFalse(rfc5424Frame.next()); + counter++; + } + + iterator = messageList.iterator(); + for (int i = 0; i <= 13; i++) { + recordOffset = messages.get(1).get(counter); + Assertions + .assertEquals( + "{\"topic\":\"testConsumerTopic\", \"partition\":\"4\", \"offset\":\"" + i + "\"}", + recordOffset.offsetToJSON() + ); + rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); + Assertions.assertTrue(rfc5424Frame.next()); + Assertions.assertTrue(iterator.hasNext()); + Assertions.assertEquals(iterator.next(), rfc5424Frame.msg.toString()); + Assertions.assertFalse(rfc5424Frame.next()); + counter++; + } + + iterator = messageList.iterator(); + for (int i = 0; i <= 13; i++) { + recordOffset = messages.get(1).get(counter); + Assertions + .assertEquals( + "{\"topic\":\"testConsumerTopic\", \"partition\":\"2\", \"offset\":\"" + i + "\"}", + recordOffset.offsetToJSON() + ); + rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); + Assertions.assertTrue(rfc5424Frame.next()); + Assertions.assertTrue(iterator.hasNext()); + Assertions.assertEquals(iterator.next(), rfc5424Frame.msg.toString()); + Assertions.assertFalse(rfc5424Frame.next()); + counter++; + } + + iterator = messageList.iterator(); + for (int i = 0; i <= 13; i++) { + recordOffset = messages.get(1).get(counter); + Assertions + .assertEquals( + "{\"topic\":\"testConsumerTopic\", \"partition\":\"0\", \"offset\":\"" + i + "\"}", + recordOffset.offsetToJSON() + ); + rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); + Assertions.assertTrue(rfc5424Frame.next()); + Assertions.assertTrue(iterator.hasNext()); + Assertions.assertEquals(iterator.next(), rfc5424Frame.msg.toString()); + Assertions.assertFalse(rfc5424Frame.next()); + counter++; + } + Assertions.assertEquals(70, counter); + }); } - @Disabled @Test - public void kafkaAndAvroFullTest() { + public void readCoordinatorTest1Thread() { assertDoesNotThrow(() -> { Config config = new Config(); - config.setMaximumFileSize(3000); // 10 loops (140 records) are in use at the moment, and that is sized at 36,102 bytes. - HdfsDataIngestion hdfsDataIngestion = new HdfsDataIngestion(config); - hdfsDataIngestion.run(); - int counter = avroReader(1, 2); - Assertions.assertEquals(140, counter); - cleanup(config, 1, 2); - }); - } + Map hdfsStartOffsets = new HashMap<>(); + ArrayList> messages = new ArrayList<>(); + Consumer> output = message -> messages.add(message); - // Reads the data from a list of avro files - public int avroReader(int start, int end) { - return assertDoesNotThrow(() -> { - // Deserialize Users from disk - Config config = new Config(); - Path queueDirectory = Paths.get(config.getQueueDirectory()); - int looper = 0; - int counter = 0; - int partitionCounter = 0; - for (int j = 0; j <= 9; j++) { - for (int i = start; i <= end; i++) { - File syslogFile = new File( - queueDirectory.toAbsolutePath() + File.separator + "testConsumerTopic" + j + "." + i + ReadCoordinator readCoordinator = new ReadCoordinator( + "testConsumerTopic", + config.getKafkaConsumerProperties(), + output, + hdfsStartOffsets + ); + Thread readThread = new Thread(null, readCoordinator, "testConsumerTopic0"); // Starts the thread with readCoordinator that creates the consumer and subscribes to the topic. + readThread.start(); // Starts the thread, in other words proceeds to call run() function of ReadCoordinator. + + Thread.sleep(10000); + Assertions.assertEquals(1, messages.size()); + Assertions.assertEquals(140, messages.get(0).size()); // Assert that expected amount of records has been consumed by the consumer. + + // Assert that all the record contents are correct, every topic partition has identical set of offset-message pairings. + List list = new ArrayList(); + list.add("[WARN] 2022-04-25 07:34:50,804 com.teragrep.jla_02.Log4j Log - Log4j warn says hi!"); + list.add("[ERROR] 2022-04-25 07:34:50,806 com.teragrep.jla_02.Log4j Log - Log4j error says hi!"); + list.add("470647 [Thread-3] INFO com.teragrep.jla_02.Logback Daily - Logback-daily says hi."); + list.add("470646 [Thread-3] INFO com.teragrep.jla_02.Logback Audit - Logback-audit says hi."); + list.add("470647 [Thread-3] INFO com.teragrep.jla_02.Logback Metric - Logback-metric says hi."); + list + .add( + "25.04.2022 07:34:52.238 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info audit says hi!]" + ); + list + .add( + "25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info daily says hi!]" + ); + list + .add( + "25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info metric says hi!]" + ); + list + .add( + "25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn audit says hi!]" + ); + list + .add( + "25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn daily says hi!]" ); - DatumReader userDatumReader = new SpecificDatumReader<>(SyslogRecord.class); - try ( - DataFileReader dataFileReader = new DataFileReader<>( - syslogFile, - userDatumReader - ) - ) { - SyslogRecord user = null; - while (dataFileReader.hasNext()) { - user = dataFileReader.next(user); - if (LOGGER.isDebugEnabled()) { - LOGGER.debug(syslogFile.getPath()); - LOGGER.debug(user.toString()); - } - counter++; - // All the mock data is generated from a set of 14 records. - if (looper <= 0) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872090804000, \"message\": \"[WARN] 2022-04-25 07:34:50,804 com.teragrep.jla_02.Log4j Log - Log4j warn says hi!\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 0, \"origin\": \"jla-02.default\"}", - user.toString() - ); - looper++; - } - else if (looper == 1) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872090806000, \"message\": \"[ERROR] 2022-04-25 07:34:50,806 com.teragrep.jla_02.Log4j Log - Log4j error says hi!\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 1, \"origin\": \"jla-02.default\"}", - user.toString() - ); - looper++; - } - else if (looper == 2) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872090822000, \"message\": \"470647 [Thread-3] INFO com.teragrep.jla_02.Logback Daily - Logback-daily says hi.\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 2, \"origin\": \"jla-02\"}", - user.toString() - ); - looper++; - } - else if (looper == 3) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872090822000, \"message\": \"470646 [Thread-3] INFO com.teragrep.jla_02.Logback Audit - Logback-audit says hi.\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 3, \"origin\": \"jla-02\"}", - user.toString() - ); - looper++; - } - else if (looper == 4) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872090822000, \"message\": \"470647 [Thread-3] INFO com.teragrep.jla_02.Logback Metric - Logback-metric says hi.\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 4, \"origin\": \"jla-02\"}", - user.toString() - ); - looper++; - } - else if (looper == 5) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872092238000, \"message\": \"25.04.2022 07:34:52.238 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info audit says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 5, \"origin\": \"jla-02.default\"}", - user.toString() - ); - looper++; - } - else if (looper == 6) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872092239000, \"message\": \"25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info daily says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 6, \"origin\": \"jla-02.default\"}", - user.toString() - ); - looper++; - } - else if (looper == 7) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872092239000, \"message\": \"25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info metric says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 7, \"origin\": \"jla-02.default\"}", - user.toString() - ); - looper++; - } - else if (looper == 8) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872092240000, \"message\": \"25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn audit says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 8, \"origin\": \"jla-02.default\"}", - user.toString() - ); - looper++; - } - else if (looper == 9) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872092240000, \"message\": \"25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn daily says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 9, \"origin\": \"jla-02.default\"}", - user.toString() - ); - looper++; - } - else if (looper == 10) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872092241000, \"message\": \"25.04.2022 07:34:52.241 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn metric says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 10, \"origin\": \"jla-02.default\"}", - user.toString() - ); - looper++; - } - else if (looper == 11) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872092241000, \"message\": \"25.04.2022 07:34:52.241 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error audit says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 11, \"origin\": \"jla-02.default\"}", - user.toString() - ); - looper++; - } - else if (looper == 12) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872092242000, \"message\": \"25.04.2022 07:34:52.242 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error daily says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 12, \"origin\": \"jla-02.default\"}", - user.toString() - ); - looper++; - } - else { - Assertions - .assertEquals( - "{\"timestamp\": 1650872092243000, \"message\": \"25.04.2022 07:34:52.243 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error metric says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 13, \"origin\": \"jla-02.default\"}", - user.toString() - ); - looper = 0; - partitionCounter++; - } - } - - } - } + list + .add( + "25.04.2022 07:34:52.241 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn metric says hi!]" + ); + list + .add( + "25.04.2022 07:34:52.241 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error audit says hi!]" + ); + list + .add( + "25.04.2022 07:34:52.242 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error daily says hi!]" + ); + list + .add( + "25.04.2022 07:34:52.243 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error metric says hi!]" + ); + + RFC5424Frame rfc5424Frame = new RFC5424Frame(false); + + RecordOffset recordOffset; + + Iterator iterator = list.iterator(); + int counter = 0; + for (int i = 0; i <= 13; i++) { + recordOffset = messages.get(0).get(counter); + Assertions + .assertEquals( + "{\"topic\":\"testConsumerTopic\", \"partition\":\"7\", \"offset\":\"" + i + "\"}", + recordOffset.offsetToJSON() + ); + rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); + Assertions.assertTrue(rfc5424Frame.next()); + Assertions.assertTrue(iterator.hasNext()); + Assertions.assertEquals(iterator.next(), rfc5424Frame.msg.toString()); + Assertions.assertFalse(rfc5424Frame.next()); + counter++; } - LOGGER.debug("Total number of records: " + counter); - return counter; - }); - } - // Deletes the avro-files that were created during testing. - public void cleanup(Config config, int start, int end) { - Path queueDirectory = Paths.get(config.getQueueDirectory()); - for (int j = 0; j <= 9; j++) { - for (int i = start; i <= end; i++) { - File syslogFile = new File( - queueDirectory.toAbsolutePath() + File.separator + "testConsumerTopic" + j + "." + i - ); - assertDoesNotThrow(() -> { - boolean result = Files.deleteIfExists(syslogFile.toPath()); - }); + iterator = list.iterator(); + for (int i = 0; i <= 13; i++) { + recordOffset = messages.get(0).get(counter); + Assertions + .assertEquals( + "{\"topic\":\"testConsumerTopic\", \"partition\":\"8\", \"offset\":\"" + i + "\"}", + recordOffset.offsetToJSON() + ); + rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); + Assertions.assertTrue(rfc5424Frame.next()); + Assertions.assertTrue(iterator.hasNext()); + Assertions.assertEquals(iterator.next(), rfc5424Frame.msg.toString()); + Assertions.assertFalse(rfc5424Frame.next()); + counter++; } - } + + iterator = list.iterator(); + for (int i = 0; i <= 13; i++) { + recordOffset = messages.get(0).get(counter); + Assertions + .assertEquals( + "{\"topic\":\"testConsumerTopic\", \"partition\":\"5\", \"offset\":\"" + i + "\"}", + recordOffset.offsetToJSON() + ); + rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); + Assertions.assertTrue(rfc5424Frame.next()); + Assertions.assertEquals(iterator.next(), rfc5424Frame.msg.toString()); + Assertions.assertFalse(rfc5424Frame.next()); + counter++; + } + + iterator = list.iterator(); + for (int i = 0; i <= 13; i++) { + recordOffset = messages.get(0).get(counter); + Assertions + .assertEquals( + "{\"topic\":\"testConsumerTopic\", \"partition\":\"6\", \"offset\":\"" + i + "\"}", + recordOffset.offsetToJSON() + ); + rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); + Assertions.assertTrue(rfc5424Frame.next()); + Assertions.assertEquals(iterator.next(), rfc5424Frame.msg.toString()); + Assertions.assertFalse(rfc5424Frame.next()); + counter++; + } + + iterator = list.iterator(); + for (int i = 0; i <= 13; i++) { + recordOffset = messages.get(0).get(counter); + Assertions + .assertEquals( + "{\"topic\":\"testConsumerTopic\", \"partition\":\"3\", \"offset\":\"" + i + "\"}", + recordOffset.offsetToJSON() + ); + rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); + Assertions.assertTrue(rfc5424Frame.next()); + Assertions.assertEquals(iterator.next(), rfc5424Frame.msg.toString()); + Assertions.assertFalse(rfc5424Frame.next()); + counter++; + } + + iterator = list.iterator(); + for (int i = 0; i <= 13; i++) { + recordOffset = messages.get(0).get(counter); + Assertions + .assertEquals( + "{\"topic\":\"testConsumerTopic\", \"partition\":\"4\", \"offset\":\"" + i + "\"}", + recordOffset.offsetToJSON() + ); + rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); + Assertions.assertTrue(rfc5424Frame.next()); + Assertions.assertEquals(iterator.next(), rfc5424Frame.msg.toString()); + Assertions.assertFalse(rfc5424Frame.next()); + counter++; + } + + iterator = list.iterator(); + for (int i = 0; i <= 13; i++) { + recordOffset = messages.get(0).get(counter); + Assertions + .assertEquals( + "{\"topic\":\"testConsumerTopic\", \"partition\":\"1\", \"offset\":\"" + i + "\"}", + recordOffset.offsetToJSON() + ); + rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); + Assertions.assertTrue(rfc5424Frame.next()); + Assertions.assertEquals(iterator.next(), rfc5424Frame.msg.toString()); + Assertions.assertFalse(rfc5424Frame.next()); + counter++; + } + + iterator = list.iterator(); + for (int i = 0; i <= 13; i++) { + recordOffset = messages.get(0).get(counter); + Assertions + .assertEquals( + "{\"topic\":\"testConsumerTopic\", \"partition\":\"2\", \"offset\":\"" + i + "\"}", + recordOffset.offsetToJSON() + ); + rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); + Assertions.assertTrue(rfc5424Frame.next()); + Assertions.assertEquals(iterator.next(), rfc5424Frame.msg.toString()); + Assertions.assertFalse(rfc5424Frame.next()); + counter++; + } + + iterator = list.iterator(); + for (int i = 0; i <= 13; i++) { + recordOffset = messages.get(0).get(counter); + Assertions + .assertEquals( + "{\"topic\":\"testConsumerTopic\", \"partition\":\"0\", \"offset\":\"" + i + "\"}", + recordOffset.offsetToJSON() + ); + rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); + Assertions.assertTrue(rfc5424Frame.next()); + Assertions.assertEquals(iterator.next(), rfc5424Frame.msg.toString()); + Assertions.assertFalse(rfc5424Frame.next()); + counter++; + } + + iterator = list.iterator(); + for (int i = 0; i <= 13; i++) { + recordOffset = messages.get(0).get(counter); + Assertions + .assertEquals( + "{\"topic\":\"testConsumerTopic\", \"partition\":\"9\", \"offset\":\"" + i + "\"}", + recordOffset.offsetToJSON() + ); + rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); + Assertions.assertTrue(rfc5424Frame.next()); + Assertions.assertEquals(iterator.next(), rfc5424Frame.msg.toString()); + Assertions.assertFalse(rfc5424Frame.next()); + counter++; + } + + Assertions.assertEquals(140, counter); // All 140 records asserted. + + }); } + } From f67e59208bb4738dd503242db41f9c475b0e23a8 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Wed, 12 Jun 2024 14:27:07 +0300 Subject: [PATCH 091/146] Refactored CombinedFullTest.java. Spotless. --- .../consumers/kafka/DatabaseOutput.java | 7 +- .../com/teragrep/cfe_39/CombinedFullTest.java | 658 +++++++++++++++--- 2 files changed, 583 insertions(+), 82 deletions(-) diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java index 6a22875f..0a651bb6 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java @@ -97,7 +97,12 @@ public class DatabaseOutput implements Consumer> { private final Config config; private long epochMicros_last; - DatabaseOutput(Config config, String table, DurationStatistics durationStatistics, TopicCounter topicCounter) { + public DatabaseOutput( + Config config, + String table, + DurationStatistics durationStatistics, + TopicCounter topicCounter + ) { this.config = config; this.table = table; this.durationStatistics = durationStatistics; diff --git a/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java b/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java index bb41c436..96453a0e 100644 --- a/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java +++ b/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java @@ -125,56 +125,601 @@ public void teardownMiniCluster() { } @Test - public void kafkaAndAvroFullTest() { + public void ingestionTest0Files() { // Empty HDFS database, 140 records in mock kafka consumer ready for ingestion. All 14 records for each 10 topic partitions are stored in a single avro-file per partition. assertDoesNotThrow(() -> { Assertions.assertTrue(config.getPruneOffset() >= 300000L); // Fails the test if the config is not correct. - config.setMaximumFileSize(30000); + config.setMaximumFileSize(30000); // This parameter defines the amount of records that can fit inside a single AVRO-file. HdfsDataIngestion hdfsDataIngestion = new HdfsDataIngestion(config); Thread.sleep(10000); hdfsDataIngestion.run(); // Assert that the kafka records were ingested correctly and the database holds the correct 140 records. - hdfsReadCheck(); + hdfsReadCheck0Files(); + }); + } + + @Test + public void ingestionTest2OldFiles() { + /* 14 records are inserted to HDFS database before starting ingestion, with 124/140 records in mock kafka consumer ready for ingestion. + Partitions through 1 to 9 will have only a single file, partition 0 will have 2 files (0.9 and 0.13). + partition 0 files are pre-made and inserted to the HDFS database with old timestamps that will mark them for pruning when ingestion is started.*/ + insertMockFiles(157784760000L, 157784760000L); // Insert 2 mock files (0.9 and 0.13) with old timestamps so pruning should trigger on them. + + assertDoesNotThrow(() -> { + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic"))); + Assertions + .assertEquals(2, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.13"))); + Assertions.assertTrue(config.getPruneOffset() >= 300000L); // Fails the test if the config is not correct. + Assertions.assertTrue((System.currentTimeMillis() - config.getPruneOffset()) > 157784760000L); + config.setMaximumFileSize(30000); + HdfsDataIngestion hdfsDataIngestion = new HdfsDataIngestion(config); + Thread.sleep(10000); + hdfsDataIngestion.run(); + + // hdfsDataIngestion.run(); has called fs.close() after finishing ingesting the records from kafka. Rebuild fs. + String hdfsURI = "hdfs://localhost:" + hdfsCluster.getNameNodePort() + "/"; + config.setHdfsuri(hdfsURI); + // ====== Init HDFS File System Object + Configuration fsConf = new Configuration(); + // Set FileSystem URI + fsConf.set("fs.defaultFS", hdfsURI); + // Because of Maven + fsConf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); + fsConf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); + // Set HADOOP user + System.setProperty("HADOOP_USER_NAME", "hdfs"); + System.setProperty("hadoop.home.dir", "/"); + fs = FileSystem.get(URI.create(hdfsURI), fsConf); + + // Assert that the kafka records were ingested and pruned correctly and the database holds only the expected 9 files. + Assertions + .assertEquals(9, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); + Assertions.assertFalse(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); + Assertions + .assertFalse(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.13"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "1.13"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "2.13"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "3.13"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "4.13"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "5.13"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "6.13"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "7.13"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "8.13"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "9.13"))); + }); + } + + @Test + public void ingestionTest1Old1NewFile() { + /* 14 records are inserted to HDFS database before starting ingestion, with 124/140 records in mock kafka consumer ready for ingestion. + Partitions through 1 to 9 will have only a single file, partition 0 will have 2 files (0.9 and 0.13). + partition 0 files are pre-made and inserted to the HDFS database with old timestamp for file 0.9 and new for 0.13.*/ + insertMockFiles(157784760000L, -1); // Insert 2 mock files (0.9 and 0.13) with old timestamp on 0.9 and new timestamp on 0.13. + + assertDoesNotThrow(() -> { + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic"))); + Assertions + .assertEquals(2, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.13"))); + Assertions.assertTrue(config.getPruneOffset() >= 300000L); // Fails the test if the config is not correct. + Assertions.assertTrue((System.currentTimeMillis() - config.getPruneOffset()) > 157784760000L); + config.setMaximumFileSize(30000); + HdfsDataIngestion hdfsDataIngestion = new HdfsDataIngestion(config); + Thread.sleep(10000); + hdfsDataIngestion.run(); + + // hdfsDataIngestion.run(); has called fs.close() after finishing ingesting the records from kafka. Rebuild fs. + String hdfsURI = "hdfs://localhost:" + hdfsCluster.getNameNodePort() + "/"; + config.setHdfsuri(hdfsURI); + // ====== Init HDFS File System Object + Configuration fsConf = new Configuration(); + // Set FileSystem URI + fsConf.set("fs.defaultFS", hdfsURI); + // Because of Maven + fsConf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); + fsConf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); + // Set HADOOP user + System.setProperty("HADOOP_USER_NAME", "hdfs"); + System.setProperty("hadoop.home.dir", "/"); + fs = FileSystem.get(URI.create(hdfsURI), fsConf); + + // Assert that the kafka records were ingested and pruned correctly and the database holds only the expected 10 files. + Assertions + .assertEquals(10, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); + Assertions.assertFalse(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.13"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "1.13"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "2.13"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "3.13"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "4.13"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "5.13"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "6.13"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "7.13"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "8.13"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "9.13"))); }); } - // TODO: Make a separate test out of this. - // 1. Insert the pre-made files into the database using insertMockFiles(). - // 2. set their timestamp to something that will be pruned. - // 3. Run HdfsDataIngestion like in kafkaAndAvroFullTest(). - // 4. Assert that only the expected files are present in the HDFS database. - public void hdfsPruneTest() { + @Test + public void ingestionTest2NewFiles() { + // 14 records are inserted to HDFS database before starting ingestion, with 124/140 records in mock kafka consumer ready for ingestion. Partitions through 1 to 9 will have only a single file, partition 0 will have 2 files (0.9 and 0.13). + insertMockFiles(-1, -1); // Insert 2 mock files (0.9 and 0.13) with new timestamps so pruning should not trigger on them. + + assertDoesNotThrow(() -> { + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic"))); + Assertions + .assertEquals(2, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.13"))); + Assertions.assertTrue(config.getPruneOffset() >= 300000L); // Fails the test if the config is not correct. + config.setMaximumFileSize(30000); + HdfsDataIngestion hdfsDataIngestion = new HdfsDataIngestion(config); + Thread.sleep(10000); + hdfsDataIngestion.run(); + + // hdfsDataIngestion.run(); has called fs.close() after finishing ingesting the records from kafka. Rebuild fs. + String hdfsURI = "hdfs://localhost:" + hdfsCluster.getNameNodePort() + "/"; + config.setHdfsuri(hdfsURI); + // ====== Init HDFS File System Object + Configuration fsConf = new Configuration(); + // Set FileSystem URI + fsConf.set("fs.defaultFS", hdfsURI); + // Because of Maven + fsConf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); + fsConf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); + // Set HADOOP user + System.setProperty("HADOOP_USER_NAME", "hdfs"); + System.setProperty("hadoop.home.dir", "/"); + fs = FileSystem.get(URI.create(hdfsURI), fsConf); + + // Assert that the kafka records were ingested correctly and the database holds the expected 11 files holding the expected 140 records. + Assertions + .assertEquals(11, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.13"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "1.13"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "2.13"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "3.13"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "4.13"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "5.13"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "6.13"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "7.13"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "8.13"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "9.13"))); + hdfsReadCheck2NewFiles(); + }); + } + + // Checks the contents of the files generated during kafkaAndAvroFullTestWithNewFiles(). + public void hdfsReadCheck2NewFiles() { + + // Check that the files were properly written to HDFS. + String hdfsuri = config.getHdfsuri(); String path = config.getHdfsPath() + "/" + "testConsumerTopic"; + // ====== Init HDFS File System Object + Configuration conf = new Configuration(); + // Set FileSystem URI + conf.set("fs.defaultFS", hdfsuri); + // Because of Maven + conf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); + conf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); + // Set HADOOP user + System.setProperty("HADOOP_USER_NAME", "hdfs"); + System.setProperty("hadoop.home.dir", "/"); //Get the filesystem - HDFS assertDoesNotThrow(() -> { + FileSystem fs = FileSystem.get(URI.create(hdfsuri), conf); Path workingDir = fs.getWorkingDirectory(); Path newDirectoryPath = new Path(path); Assertions.assertTrue(fs.exists(newDirectoryPath)); - /* Use HDFS-file modification timestamps for pruning. - fs.setTimes(new Path(path+"/"+0.8), Long.parseUnsignedLong("1675930598000"), -1); - where mtime is modification time and atime is access time. -1 as input parameter leaves the original atime/mtime value as is.*/ - FileStatus[] fileStatuses = fs.listStatus(new Path(newDirectoryPath + "/")); - long count = Arrays.stream(fileStatuses).count(); - Assertions.assertTrue(count > 0); - - for (FileStatus a : fileStatuses) { - // Delete old files - if (a.getModificationTime() < System.currentTimeMillis()) { - boolean deleted = fs.delete(a.getPath(), true); - Assertions.assertTrue(deleted); - LOGGER.info("Deleted file {}", a.getPath()); - LOGGER.debug("Timestamp was {}", a.getModificationTime()); // example: 1717582386630 milliseconds which translates to Wednesday, June 5, 2024 1:13:06.630 PM GMT+03:00 DST - } + /* This is the HDFS write path for the files: + Path hdfswritepath = new Path(newDirectoryPath + "/" + fileName); where newDirectoryPath is config.getHdfsPath() + "/" + lastObject.topic; and filename is lastObject.partition+"."+lastObject.offset; + + Create the list of files to read from HDFS. Test setup is created so each of the 1-9 partitions will have 1 file with offset of 13, while the 0th partition will have 2 files with offset 9 and 13.*/ + List filenameList = new ArrayList<>(); + filenameList.add("0.9"); + filenameList.add("0.13"); + for (int i = 1; i <= 9; i++) { + filenameList.add(i + "." + 13); + } + FileStatus[] fileStatuses = fs.listStatus(newDirectoryPath); + Assertions.assertEquals(filenameList.size(), fileStatuses.length); + for (FileStatus fileStatus : fileStatuses) { + Assertions.assertTrue(filenameList.contains(fileStatus.getPath().getName())); + } + LOGGER.info("All expected files present in HDFS."); + + int partitionCounter = 0; + + // Assertions for file testConsumerTopic/0.9 + String fileName0 = filenameList.get(0); + Assertions.assertEquals("0.9", fileName0); + // Assert that file testConsumerTopic/0.9 has expected content. + LOGGER.info("Read file into hdfs"); + //Create a path + Path hdfsreadpath = new Path(newDirectoryPath + "/" + fileName0); // The path should be the same that was used in writing the file to HDFS. + //Init input stream + FSDataInputStream inputStream = fs.open(hdfsreadpath); + //The data is in AVRO-format, so it can't be read as a string. + DataFileStream reader = new DataFileStream<>( + inputStream, + new SpecificDatumReader<>(SyslogRecord.class) + ); + SyslogRecord record = null; + LOGGER.info("\nReading records from file {}:", hdfsreadpath); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug(record.toString()); + } + Assertions + .assertEquals( + "{\"timestamp\": 1650872090804000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 0, \"origin\": \"jla-02.default\", \"payload\": \"[WARN] 2022-04-25 07:34:50,804 com.teragrep.jla_02.Log4j Log - Log4j warn says hi!\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug(record.toString()); + } + Assertions + .assertEquals( + "{\"timestamp\": 1650872090806000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 1, \"origin\": \"jla-02.default\", \"payload\": \"[ERROR] 2022-04-25 07:34:50,806 com.teragrep.jla_02.Log4j Log - Log4j error says hi!\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug(record.toString()); + } + Assertions + .assertEquals( + "{\"timestamp\": 1650872090822000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 2, \"origin\": \"jla-02\", \"payload\": \"470647 [Thread-3] INFO com.teragrep.jla_02.Logback Daily - Logback-daily says hi.\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug(record.toString()); } - LOGGER.info("All files were pruned properly."); + Assertions + .assertEquals( + "{\"timestamp\": 1650872090822000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 3, \"origin\": \"jla-02\", \"payload\": \"470646 [Thread-3] INFO com.teragrep.jla_02.Logback Audit - Logback-audit says hi.\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug(record.toString()); + } + Assertions + .assertEquals( + "{\"timestamp\": 1650872090822000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 4, \"origin\": \"jla-02\", \"payload\": \"470647 [Thread-3] INFO com.teragrep.jla_02.Logback Metric - Logback-metric says hi.\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug(record.toString()); + } + Assertions + .assertEquals( + "{\"timestamp\": 1650872092238000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 5, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.238 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info audit says hi!]\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug(record.toString()); + } + Assertions + .assertEquals( + "{\"timestamp\": 1650872092239000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 6, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info daily says hi!]\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug(record.toString()); + } + Assertions + .assertEquals( + "{\"timestamp\": 1650872092239000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 7, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info metric says hi!]\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug(record.toString()); + } + Assertions + .assertEquals( + "{\"timestamp\": 1650872092240000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 8, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn audit says hi!]\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug(record.toString()); + } + Assertions + .assertEquals( + "{\"timestamp\": 1650872092240000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 9, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn daily says hi!]\"}", + record.toString() + ); + + Assertions.assertFalse(reader.hasNext()); // Reached the end of the testConsumerTopic/0.9 file. + inputStream.close(); + filenameList.remove(0); + + // Assertions for file testConsumerTopic/0.13 + fileName0 = filenameList.get(0); + Assertions.assertEquals("0.13", fileName0); + LOGGER.info("Read file into hdfs"); + //Create a path + hdfsreadpath = new Path(newDirectoryPath + "/" + fileName0); // The path should be the same that was used in writing the file to HDFS. + //Init input stream + inputStream = fs.open(hdfsreadpath); + //The data is in AVRO-format, so it can't be read as a string. + reader = new DataFileStream<>(inputStream, new SpecificDatumReader<>(SyslogRecord.class)); + record = null; + LOGGER.info("\nReading records from file {}:", hdfsreadpath); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug(record.toString()); + } + Assertions + .assertEquals( + "{\"timestamp\": 1650872092241000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 10, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.241 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn metric says hi!]\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug(record.toString()); + } + Assertions + .assertEquals( + "{\"timestamp\": 1650872092241000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 11, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.241 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error audit says hi!]\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug(record.toString()); + } + Assertions + .assertEquals( + "{\"timestamp\": 1650872092242000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 12, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.242 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error daily says hi!]\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug(record.toString()); + } + Assertions + .assertEquals( + "{\"timestamp\": 1650872092243000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 13, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.243 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error metric says hi!]\"}", + record.toString() + ); + Assertions.assertFalse(reader.hasNext()); // Reached the end of the testConsumerTopic/0.13 file. + inputStream.close(); + filenameList.remove(0); + + partitionCounter++; + + for (String fileName : filenameList) { + //==== Read files + LOGGER.info("Read file into hdfs"); + //Create a path + hdfsreadpath = new Path(newDirectoryPath + "/" + fileName); // The path should be the same that was used in writing the file to HDFS. + //Init input stream + inputStream = fs.open(hdfsreadpath); + //The data is in AVRO-format, so it can't be read as a string. + reader = new DataFileStream<>(inputStream, new SpecificDatumReader<>(SyslogRecord.class)); + record = null; + LOGGER.info("\nReading records from file {}:", hdfsreadpath); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + Assertions + .assertEquals( + "{\"timestamp\": 1650872090804000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 0, \"origin\": \"jla-02.default\", \"payload\": \"[WARN] 2022-04-25 07:34:50,804 com.teragrep.jla_02.Log4j Log - Log4j warn says hi!\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + Assertions + .assertEquals( + "{\"timestamp\": 1650872090806000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 1, \"origin\": \"jla-02.default\", \"payload\": \"[ERROR] 2022-04-25 07:34:50,806 com.teragrep.jla_02.Log4j Log - Log4j error says hi!\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + Assertions + .assertEquals( + "{\"timestamp\": 1650872090822000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 2, \"origin\": \"jla-02\", \"payload\": \"470647 [Thread-3] INFO com.teragrep.jla_02.Logback Daily - Logback-daily says hi.\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + Assertions + .assertEquals( + "{\"timestamp\": 1650872090822000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 3, \"origin\": \"jla-02\", \"payload\": \"470646 [Thread-3] INFO com.teragrep.jla_02.Logback Audit - Logback-audit says hi.\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + Assertions + .assertEquals( + "{\"timestamp\": 1650872090822000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 4, \"origin\": \"jla-02\", \"payload\": \"470647 [Thread-3] INFO com.teragrep.jla_02.Logback Metric - Logback-metric says hi.\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + Assertions + .assertEquals( + "{\"timestamp\": 1650872092238000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 5, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.238 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info audit says hi!]\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + Assertions + .assertEquals( + "{\"timestamp\": 1650872092239000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 6, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info daily says hi!]\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + Assertions + .assertEquals( + "{\"timestamp\": 1650872092239000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 7, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info metric says hi!]\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + Assertions + .assertEquals( + "{\"timestamp\": 1650872092240000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 8, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn audit says hi!]\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + Assertions + .assertEquals( + "{\"timestamp\": 1650872092240000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 9, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn daily says hi!]\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + Assertions + .assertEquals( + "{\"timestamp\": 1650872092241000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 10, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.241 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn metric says hi!]\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + Assertions + .assertEquals( + "{\"timestamp\": 1650872092241000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 11, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.241 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error audit says hi!]\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + Assertions + .assertEquals( + "{\"timestamp\": 1650872092242000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 12, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.242 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error daily says hi!]\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + Assertions + .assertEquals( + "{\"timestamp\": 1650872092243000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 13, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.243 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error metric says hi!]\"}", + record.toString() + ); + Assertions.assertFalse(reader.hasNext()); + LOGGER.info("Partition {} passed assertions.", partitionCounter); + partitionCounter++; + inputStream.close(); + } + Assertions.assertEquals(10, partitionCounter); }); } // This method checks that the expected files with expected contents are present in the HDFS database. - public void hdfsReadCheck() { + public void hdfsReadCheck0Files() { // Check that the files were properly written to HDFS. String hdfsuri = config.getHdfsuri(); @@ -230,9 +775,6 @@ public void hdfsReadCheck() { Assertions.assertTrue(reader.hasNext()); record = reader.next(record); - if (LOGGER.isDebugEnabled()) { - LOGGER.debug(record.toString()); - } Assertions .assertEquals( "{\"timestamp\": 1650872090804000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" @@ -243,9 +785,6 @@ record = reader.next(record); Assertions.assertTrue(reader.hasNext()); record = reader.next(record); - if (LOGGER.isDebugEnabled()) { - LOGGER.debug(record.toString()); - } Assertions .assertEquals( "{\"timestamp\": 1650872090806000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" @@ -256,9 +795,6 @@ record = reader.next(record); Assertions.assertTrue(reader.hasNext()); record = reader.next(record); - if (LOGGER.isDebugEnabled()) { - LOGGER.debug(record.toString()); - } Assertions .assertEquals( "{\"timestamp\": 1650872090822000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" @@ -269,9 +805,6 @@ record = reader.next(record); Assertions.assertTrue(reader.hasNext()); record = reader.next(record); - if (LOGGER.isDebugEnabled()) { - LOGGER.debug(record.toString()); - } Assertions .assertEquals( "{\"timestamp\": 1650872090822000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" @@ -282,9 +815,6 @@ record = reader.next(record); Assertions.assertTrue(reader.hasNext()); record = reader.next(record); - if (LOGGER.isDebugEnabled()) { - LOGGER.debug(record.toString()); - } Assertions .assertEquals( "{\"timestamp\": 1650872090822000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" @@ -295,9 +825,6 @@ record = reader.next(record); Assertions.assertTrue(reader.hasNext()); record = reader.next(record); - if (LOGGER.isDebugEnabled()) { - LOGGER.debug(record.toString()); - } Assertions .assertEquals( "{\"timestamp\": 1650872092238000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" @@ -308,9 +835,6 @@ record = reader.next(record); Assertions.assertTrue(reader.hasNext()); record = reader.next(record); - if (LOGGER.isDebugEnabled()) { - LOGGER.debug(record.toString()); - } Assertions .assertEquals( "{\"timestamp\": 1650872092239000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" @@ -321,9 +845,6 @@ record = reader.next(record); Assertions.assertTrue(reader.hasNext()); record = reader.next(record); - if (LOGGER.isDebugEnabled()) { - LOGGER.debug(record.toString()); - } Assertions .assertEquals( "{\"timestamp\": 1650872092239000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" @@ -334,9 +855,6 @@ record = reader.next(record); Assertions.assertTrue(reader.hasNext()); record = reader.next(record); - if (LOGGER.isDebugEnabled()) { - LOGGER.debug(record.toString()); - } Assertions .assertEquals( "{\"timestamp\": 1650872092240000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" @@ -347,9 +865,6 @@ record = reader.next(record); Assertions.assertTrue(reader.hasNext()); record = reader.next(record); - if (LOGGER.isDebugEnabled()) { - LOGGER.debug(record.toString()); - } Assertions .assertEquals( "{\"timestamp\": 1650872092240000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" @@ -360,9 +875,6 @@ record = reader.next(record); Assertions.assertTrue(reader.hasNext()); record = reader.next(record); - if (LOGGER.isDebugEnabled()) { - LOGGER.debug(record.toString()); - } Assertions .assertEquals( "{\"timestamp\": 1650872092241000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" @@ -373,9 +885,6 @@ record = reader.next(record); Assertions.assertTrue(reader.hasNext()); record = reader.next(record); - if (LOGGER.isDebugEnabled()) { - LOGGER.debug(record.toString()); - } Assertions .assertEquals( "{\"timestamp\": 1650872092241000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" @@ -386,9 +895,6 @@ record = reader.next(record); Assertions.assertTrue(reader.hasNext()); record = reader.next(record); - if (LOGGER.isDebugEnabled()) { - LOGGER.debug(record.toString()); - } Assertions .assertEquals( "{\"timestamp\": 1650872092242000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" @@ -399,9 +905,6 @@ record = reader.next(record); Assertions.assertTrue(reader.hasNext()); record = reader.next(record); - if (LOGGER.isDebugEnabled()) { - LOGGER.debug(record.toString()); - } Assertions .assertEquals( "{\"timestamp\": 1650872092243000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" @@ -419,13 +922,11 @@ record = reader.next(record); } // Inserts pre-made avro-files to HDFS, which are normally generated during data ingestion from mock kafka consumer. - private void insertMockFiles(long a, long b) { + private void insertMockFiles(long fileTimestampA, long fileTimestampB) { String path = config.getHdfsPath() + "/" + "testConsumerTopic"; // "hdfs:///opt/teragrep/cfe_39/srv/testConsumerTopic" //Get the filesystem - HDFS assertDoesNotThrow(() -> { - //==== Create directory if not exists - Path workingDir = fs.getWorkingDirectory(); // Sets the directory where the data should be stored, if the directory doesn't exist then it's created. Path newDirectoryPath = new Path(path); if (!fs.exists(newDirectoryPath)) { @@ -448,25 +949,20 @@ private void insertMockFiles(long a, long b) { LOGGER.debug("Begin Write file into hdfs"); //Create a path Path hdfswritepath = new Path(newDirectoryPath + "/" + avroFile.getName()); // filename should be set according to the requirements: 0.12345 where 0 is Kafka partition and 12345 is Kafka offset. - if (fs.exists(hdfswritepath)) { - Assertions.fail("File " + avroFile.getName() + " already exists"); - } + Assertions.assertFalse(fs.exists(hdfswritepath)); Path readPath = new Path(avroFile.getPath()); - // Add conditions if file filtering is required for tests. fs.copyFromLocalFile(readPath, hdfswritepath); - // Set a/b to something like 157784760000 to trigger pruning. + // Set fileTimestampA/fileTimestampB to something like 157784760000 to trigger pruning, -1 to not alter the timestamp. if (Objects.equals(hdfswritepath.toString(), "hdfs:/opt/teragrep/cfe_39/srv/testConsumerTopic/0.9")) { - fs.setTimes(hdfswritepath, a, -1); + fs.setTimes(hdfswritepath, fileTimestampA, -1); } else if ( Objects.equals(hdfswritepath.toString(), "hdfs:/opt/teragrep/cfe_39/srv/testConsumerTopic/0.13") ) { - fs.setTimes(hdfswritepath, b, -1); + fs.setTimes(hdfswritepath, fileTimestampB, -1); } LOGGER.debug("End Write file into hdfs"); - if (LOGGER.isDebugEnabled()) { - LOGGER.debug("\nFile committed to HDFS, file writepath should be: {}\n", hdfswritepath.toString()); - } + LOGGER.debug("\nFile committed to HDFS, file writepath should be: {}\n", hdfswritepath); } }); } From 7d1ad2fa39b0c8ad947e9f7427b380aabf1b4c61 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Thu, 13 Jun 2024 09:37:52 +0300 Subject: [PATCH 092/146] Removed unused input arguments and parameters from HDFSWrite.java and DatabaseOutput.java. --- .../consumers/kafka/DatabaseOutput.java | 30 ++++++------------- .../cfe_39/consumers/kafka/HDFSWrite.java | 4 +-- 2 files changed, 11 insertions(+), 23 deletions(-) diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java index 0a651bb6..7fec9c70 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java @@ -95,7 +95,6 @@ public class DatabaseOutput implements Consumer> { private final SDVector originHostname; private File syslogFile; private final Config config; - private long epochMicros_last; public DatabaseOutput( Config config, @@ -122,11 +121,10 @@ public DatabaseOutput( this.eventNodeSourceHostname = new SDVector("event_node_source@48577", "hostname"); this.eventNodeRelayHostname = new SDVector("event_node_relay@48577", "hostname"); this.originHostname = new SDVector("origin@48577", "hostname"); - this.epochMicros_last = 0L; } // Checks that the filesize stays under the defined maximum file size. If the file is about to go over target limit commits the file to HDFS and returns true, otherwise does nothing and returns false. - boolean committedToHdfs(long fileSize, RecordOffset recordOffsetObject, JsonObject recordOffsetObjectJo) { + boolean committedToHdfs(long fileSize, JsonObject recordOffsetObjectJo) { try { // If the syslogAvroWriter is already initialized, check the filesize so it doesn't go above maximumFileSize. if (fileSize > maximumFileSize) { @@ -134,8 +132,8 @@ boolean committedToHdfs(long fileSize, RecordOffset recordOffsetObject, JsonObje // This part closes the writing of now "complete" AVRO-file and stores the file to HDFS. syslogAvroWriter.close(); - try (HDFSWrite writer = new HDFSWrite(config, recordOffsetObject, recordOffsetObjectJo)) { - writer.commit(syslogFile, epochMicros_last); // commits the final AVRO-file to HDFS. + try (HDFSWrite writer = new HDFSWrite(config, recordOffsetObjectJo)) { + writer.commit(syslogFile); // commits the final AVRO-file to HDFS. } // This part defines a new empty file to which the new AVRO-serialized records are stored until it again hits the 64M size limit. @@ -217,9 +215,7 @@ public void accept(List recordOffsetObjectList) { & lastObjectJo.get("partition").getAsString().equals(recordOffsetObjectJo.get("partition").getAsString()) ) { // Records left to consume in the current partition. - boolean fileCommitted = committedToHdfs( - syslogAvroWriter.getFileSize(), (RecordOffset) lastObject, lastObjectJo - ); + boolean fileCommitted = committedToHdfs(syslogAvroWriter.getFileSize(), lastObjectJo); if (fileCommitted) { if (LOGGER.isDebugEnabled()) { LOGGER @@ -243,8 +239,8 @@ public void accept(List recordOffsetObjectList) { else { // Previous partition was fully consumed. Commit file to HDFS and create a new AVRO-file. syslogAvroWriter.close(); - HDFSWrite writer = new HDFSWrite(config, (RecordOffset) lastObject, lastObjectJo); - writer.commit(syslogFile, epochMicros_last); + HDFSWrite writer = new HDFSWrite(config, lastObjectJo); + writer.commit(syslogFile); // This part defines a new empty file to which the new AVRO-serialized records are stored until it again hits the 64M size limit. writableQueue @@ -294,14 +290,8 @@ public void accept(List recordOffsetObjectList) { // Calculate the size of syslogRecord that is going to be written to syslogAvroWriter-file. long capacity = syslogRecord.toByteBuffer().capacity(); - // handle initial timestamp - if (epochMicros_last == 0L) { - epochMicros_last = epochMicros; - } // Check if there is still room in syslogAvroWriter for another syslogRecord. Commit syslogAvroWriter to HDFS if no room left, emptying it out in the process. - boolean fileCommitted = committedToHdfs( - syslogAvroWriter.getFileSize() + capacity, (RecordOffset) lastObject, lastObjectJo - ); + boolean fileCommitted = committedToHdfs(syslogAvroWriter.getFileSize() + capacity, lastObjectJo); if (fileCommitted) { if (LOGGER.isDebugEnabled()) { LOGGER @@ -321,8 +311,6 @@ public void accept(List recordOffsetObjectList) { ); } } - // if more records can be inserted, update epochMicros_last with the timestamp of the last inserted record. - epochMicros_last = epochMicros; // Add syslogRecord to syslogAvroWriter which has room for new syslogRecord. syslogAvroWriter.write(syslogRecord); lastObject = recordOffsetObject; @@ -338,8 +326,8 @@ public void accept(List recordOffsetObjectList) { try { if (syslogAvroWriter != null && !lastObject.isNull()) { syslogAvroWriter.close(); - try (HDFSWrite writer = new HDFSWrite(config, (RecordOffset) lastObject, lastObjectJo)) { - writer.commit(syslogFile, epochMicros_last); // commits the final AVRO-file to HDFS. + try (HDFSWrite writer = new HDFSWrite(config, lastObjectJo)) { + writer.commit(syslogFile); // commits the final AVRO-file to HDFS. } } } diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java index 74d5a99b..67debe63 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java @@ -69,7 +69,7 @@ public class HDFSWrite implements AutoCloseable { private final Configuration conf; private final String hdfsuri; - public HDFSWrite(Config config, RecordOffset lastObject, JsonObject lastObjectJo) throws IOException { + public HDFSWrite(Config config, JsonObject lastObjectJo) throws IOException { Properties readerKafkaProperties = config.getKafkaConsumerProperties(); this.useMockKafkaConsumer = Boolean @@ -141,7 +141,7 @@ These values should be fetched from config and other input parameters (topic+par } // Method for committing the AVRO-file to HDFS - public void commit(File syslogFile, long lastEpochMicros) { + public void commit(File syslogFile) { // The code for writing the file to HDFS should be same for both test (non-kerberized access) and prod (kerberized access). try { //==== Create directory if not exists From 068419a5895b67688a3a3f881d36203cec196e5d Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Thu, 13 Jun 2024 15:29:27 +0300 Subject: [PATCH 093/146] Removed assertion helper functions from CombinedFullTest.java. --- .../com/teragrep/cfe_39/CombinedFullTest.java | 409 +++++++++--------- 1 file changed, 200 insertions(+), 209 deletions(-) diff --git a/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java b/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java index 96453a0e..ba055f60 100644 --- a/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java +++ b/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java @@ -134,7 +134,206 @@ public void ingestionTest0Files() { Thread.sleep(10000); hdfsDataIngestion.run(); // Assert that the kafka records were ingested correctly and the database holds the correct 140 records. - hdfsReadCheck0Files(); + }); + + // Check that the files were properly written to HDFS. + String hdfsuri = config.getHdfsuri(); + + String path = config.getHdfsPath() + "/" + "testConsumerTopic"; + // ====== Init HDFS File System Object + Configuration conf = new Configuration(); + // Set FileSystem URI + conf.set("fs.defaultFS", hdfsuri); + // Because of Maven + conf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); + conf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); + // Set HADOOP user + System.setProperty("HADOOP_USER_NAME", "hdfs"); + System.setProperty("hadoop.home.dir", "/"); + //Get the filesystem - HDFS + assertDoesNotThrow(() -> { + FileSystem fs = FileSystem.get(URI.create(hdfsuri), conf); + + Path workingDir = fs.getWorkingDirectory(); + Path newDirectoryPath = new Path(path); + Assertions.assertTrue(fs.exists(newDirectoryPath)); + + /* This is the HDFS write path for the files: + Path hdfswritepath = new Path(newDirectoryPath + "/" + fileName); where newDirectoryPath is config.getHdfsPath() + "/" + lastObject.topic; and filename is lastObject.partition+"."+lastObject.offset; + + Create the list of files to read from HDFS. Test setup is created so each of the 0-9 partitions will have 1 file with offset of 13.*/ + List filenameList = new ArrayList<>(); + for (int i = 0; i <= 9; i++) { + filenameList.add(i + "." + 13); + } + FileStatus[] fileStatuses = fs.listStatus(newDirectoryPath); + Assertions.assertEquals(filenameList.size(), fileStatuses.length); + for (FileStatus fileStatus : fileStatuses) { + Assertions.assertTrue(filenameList.contains(fileStatus.getPath().getName())); + } + LOGGER.debug("All expected files present in HDFS."); + + int partitionCounter = 0; + for (String fileName : filenameList) { + //==== Read files + LOGGER.info("Read file into hdfs"); + //Create a path + Path hdfsreadpath = new Path(newDirectoryPath + "/" + fileName); // The path should be the same that was used in writing the file to HDFS. + //Init input stream + FSDataInputStream inputStream = fs.open(hdfsreadpath); + //The data is in AVRO-format, so it can't be read as a string. + DataFileStream reader = new DataFileStream<>( + inputStream, + new SpecificDatumReader<>(SyslogRecord.class) + ); + SyslogRecord record = null; + LOGGER.info("\nReading records from file {}:", hdfsreadpath); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + Assertions + .assertEquals( + "{\"timestamp\": 1650872090804000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 0, \"origin\": \"jla-02.default\", \"payload\": \"[WARN] 2022-04-25 07:34:50,804 com.teragrep.jla_02.Log4j Log - Log4j warn says hi!\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + Assertions + .assertEquals( + "{\"timestamp\": 1650872090806000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 1, \"origin\": \"jla-02.default\", \"payload\": \"[ERROR] 2022-04-25 07:34:50,806 com.teragrep.jla_02.Log4j Log - Log4j error says hi!\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + Assertions + .assertEquals( + "{\"timestamp\": 1650872090822000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 2, \"origin\": \"jla-02\", \"payload\": \"470647 [Thread-3] INFO com.teragrep.jla_02.Logback Daily - Logback-daily says hi.\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + Assertions + .assertEquals( + "{\"timestamp\": 1650872090822000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 3, \"origin\": \"jla-02\", \"payload\": \"470646 [Thread-3] INFO com.teragrep.jla_02.Logback Audit - Logback-audit says hi.\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + Assertions + .assertEquals( + "{\"timestamp\": 1650872090822000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 4, \"origin\": \"jla-02\", \"payload\": \"470647 [Thread-3] INFO com.teragrep.jla_02.Logback Metric - Logback-metric says hi.\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + Assertions + .assertEquals( + "{\"timestamp\": 1650872092238000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 5, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.238 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info audit says hi!]\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + Assertions + .assertEquals( + "{\"timestamp\": 1650872092239000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 6, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info daily says hi!]\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + Assertions + .assertEquals( + "{\"timestamp\": 1650872092239000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 7, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info metric says hi!]\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + Assertions + .assertEquals( + "{\"timestamp\": 1650872092240000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 8, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn audit says hi!]\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + Assertions + .assertEquals( + "{\"timestamp\": 1650872092240000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 9, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn daily says hi!]\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + Assertions + .assertEquals( + "{\"timestamp\": 1650872092241000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 10, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.241 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn metric says hi!]\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + Assertions + .assertEquals( + "{\"timestamp\": 1650872092241000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 11, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.241 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error audit says hi!]\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + Assertions + .assertEquals( + "{\"timestamp\": 1650872092242000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 12, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.242 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error daily says hi!]\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + Assertions + .assertEquals( + "{\"timestamp\": 1650872092243000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 13, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.243 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error metric says hi!]\"}", + record.toString() + ); + Assertions.assertFalse(reader.hasNext()); + LOGGER.info("Partition {} passed assertions.", partitionCounter); + partitionCounter++; + inputStream.close(); + } + Assertions.assertEquals(10, partitionCounter); }); } @@ -289,12 +488,7 @@ public void ingestionTest2NewFiles() { Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "7.13"))); Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "8.13"))); Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "9.13"))); - hdfsReadCheck2NewFiles(); }); - } - - // Checks the contents of the files generated during kafkaAndAvroFullTestWithNewFiles(). - public void hdfsReadCheck2NewFiles() { // Check that the files were properly written to HDFS. String hdfsuri = config.getHdfsuri(); @@ -718,209 +912,6 @@ record = reader.next(record); }); } - // This method checks that the expected files with expected contents are present in the HDFS database. - public void hdfsReadCheck0Files() { - // Check that the files were properly written to HDFS. - String hdfsuri = config.getHdfsuri(); - - String path = config.getHdfsPath() + "/" + "testConsumerTopic"; - // ====== Init HDFS File System Object - Configuration conf = new Configuration(); - // Set FileSystem URI - conf.set("fs.defaultFS", hdfsuri); - // Because of Maven - conf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); - conf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); - // Set HADOOP user - System.setProperty("HADOOP_USER_NAME", "hdfs"); - System.setProperty("hadoop.home.dir", "/"); - //Get the filesystem - HDFS - assertDoesNotThrow(() -> { - FileSystem fs = FileSystem.get(URI.create(hdfsuri), conf); - - Path workingDir = fs.getWorkingDirectory(); - Path newDirectoryPath = new Path(path); - Assertions.assertTrue(fs.exists(newDirectoryPath)); - - /* This is the HDFS write path for the files: - Path hdfswritepath = new Path(newDirectoryPath + "/" + fileName); where newDirectoryPath is config.getHdfsPath() + "/" + lastObject.topic; and filename is lastObject.partition+"."+lastObject.offset; - - Create the list of files to read from HDFS. Test setup is created so each of the 0-9 partitions will have 1 file with offset of 13.*/ - List filenameList = new ArrayList<>(); - for (int i = 0; i <= 9; i++) { - filenameList.add(i + "." + 13); - } - FileStatus[] fileStatuses = fs.listStatus(newDirectoryPath); - Assertions.assertEquals(filenameList.size(), fileStatuses.length); - for (FileStatus fileStatus : fileStatuses) { - Assertions.assertTrue(filenameList.contains(fileStatus.getPath().getName())); - } - LOGGER.debug("All expected files present in HDFS."); - - int partitionCounter = 0; - for (String fileName : filenameList) { - //==== Read files - LOGGER.info("Read file into hdfs"); - //Create a path - Path hdfsreadpath = new Path(newDirectoryPath + "/" + fileName); // The path should be the same that was used in writing the file to HDFS. - //Init input stream - FSDataInputStream inputStream = fs.open(hdfsreadpath); - //The data is in AVRO-format, so it can't be read as a string. - DataFileStream reader = new DataFileStream<>( - inputStream, - new SpecificDatumReader<>(SyslogRecord.class) - ); - SyslogRecord record = null; - LOGGER.info("\nReading records from file {}:", hdfsreadpath); - - Assertions.assertTrue(reader.hasNext()); - record = reader.next(record); - Assertions - .assertEquals( - "{\"timestamp\": 1650872090804000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 0, \"origin\": \"jla-02.default\", \"payload\": \"[WARN] 2022-04-25 07:34:50,804 com.teragrep.jla_02.Log4j Log - Log4j warn says hi!\"}", - record.toString() - ); - - Assertions.assertTrue(reader.hasNext()); - record = reader.next(record); - Assertions - .assertEquals( - "{\"timestamp\": 1650872090806000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 1, \"origin\": \"jla-02.default\", \"payload\": \"[ERROR] 2022-04-25 07:34:50,806 com.teragrep.jla_02.Log4j Log - Log4j error says hi!\"}", - record.toString() - ); - - Assertions.assertTrue(reader.hasNext()); - record = reader.next(record); - Assertions - .assertEquals( - "{\"timestamp\": 1650872090822000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 2, \"origin\": \"jla-02\", \"payload\": \"470647 [Thread-3] INFO com.teragrep.jla_02.Logback Daily - Logback-daily says hi.\"}", - record.toString() - ); - - Assertions.assertTrue(reader.hasNext()); - record = reader.next(record); - Assertions - .assertEquals( - "{\"timestamp\": 1650872090822000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 3, \"origin\": \"jla-02\", \"payload\": \"470646 [Thread-3] INFO com.teragrep.jla_02.Logback Audit - Logback-audit says hi.\"}", - record.toString() - ); - - Assertions.assertTrue(reader.hasNext()); - record = reader.next(record); - Assertions - .assertEquals( - "{\"timestamp\": 1650872090822000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 4, \"origin\": \"jla-02\", \"payload\": \"470647 [Thread-3] INFO com.teragrep.jla_02.Logback Metric - Logback-metric says hi.\"}", - record.toString() - ); - - Assertions.assertTrue(reader.hasNext()); - record = reader.next(record); - Assertions - .assertEquals( - "{\"timestamp\": 1650872092238000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 5, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.238 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info audit says hi!]\"}", - record.toString() - ); - - Assertions.assertTrue(reader.hasNext()); - record = reader.next(record); - Assertions - .assertEquals( - "{\"timestamp\": 1650872092239000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 6, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info daily says hi!]\"}", - record.toString() - ); - - Assertions.assertTrue(reader.hasNext()); - record = reader.next(record); - Assertions - .assertEquals( - "{\"timestamp\": 1650872092239000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 7, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info metric says hi!]\"}", - record.toString() - ); - - Assertions.assertTrue(reader.hasNext()); - record = reader.next(record); - Assertions - .assertEquals( - "{\"timestamp\": 1650872092240000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 8, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn audit says hi!]\"}", - record.toString() - ); - - Assertions.assertTrue(reader.hasNext()); - record = reader.next(record); - Assertions - .assertEquals( - "{\"timestamp\": 1650872092240000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 9, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn daily says hi!]\"}", - record.toString() - ); - - Assertions.assertTrue(reader.hasNext()); - record = reader.next(record); - Assertions - .assertEquals( - "{\"timestamp\": 1650872092241000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 10, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.241 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn metric says hi!]\"}", - record.toString() - ); - - Assertions.assertTrue(reader.hasNext()); - record = reader.next(record); - Assertions - .assertEquals( - "{\"timestamp\": 1650872092241000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 11, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.241 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error audit says hi!]\"}", - record.toString() - ); - - Assertions.assertTrue(reader.hasNext()); - record = reader.next(record); - Assertions - .assertEquals( - "{\"timestamp\": 1650872092242000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 12, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.242 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error daily says hi!]\"}", - record.toString() - ); - - Assertions.assertTrue(reader.hasNext()); - record = reader.next(record); - Assertions - .assertEquals( - "{\"timestamp\": 1650872092243000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 13, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.243 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error metric says hi!]\"}", - record.toString() - ); - Assertions.assertFalse(reader.hasNext()); - LOGGER.info("Partition {} passed assertions.", partitionCounter); - partitionCounter++; - inputStream.close(); - } - Assertions.assertEquals(10, partitionCounter); - }); - } - // Inserts pre-made avro-files to HDFS, which are normally generated during data ingestion from mock kafka consumer. private void insertMockFiles(long fileTimestampA, long fileTimestampB) { String path = config.getHdfsPath() + "/" + "testConsumerTopic"; // "hdfs:///opt/teragrep/cfe_39/srv/testConsumerTopic" From 62354239c6520133cea583da8adda051da3117e6 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Thu, 13 Jun 2024 16:00:02 +0300 Subject: [PATCH 094/146] Refactoring HdfsTest.java and improved HDFSWrite.java exception handling. --- .../cfe_39/consumers/kafka/HDFSWrite.java | 8 +- .../java/com/teragrep/cfe_39/HdfsTest.java | 386 ++++++------------ 2 files changed, 122 insertions(+), 272 deletions(-) diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java index 67debe63..d5ad57f8 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java @@ -159,10 +159,16 @@ public void commit(File syslogFile) { //Create a path Path hdfswritepath = new Path(newDirectoryPath.toString() + "/" + fileName); // filename should be set according to the requirements: 0.12345 where 0 is Kafka partition and 12345 is Kafka offset. if (fs.exists(hdfswritepath)) { + LOGGER + .debug( + "Deleting the seemingly duplicate source file {} because target file {} already exists in HDFS", + syslogFile.getPath(), hdfswritepath + ); + syslogFile.delete(); throw new RuntimeException("File " + fileName + " already exists"); } else { - LOGGER.info("Path <{}> doesn't exist.", path); + LOGGER.debug("Target file <{}> doesn't exist, proceeding normally.", hdfswritepath); } Path path = new Path(syslogFile.getPath()); diff --git a/src/test/java/com/teragrep/cfe_39/HdfsTest.java b/src/test/java/com/teragrep/cfe_39/HdfsTest.java index 5ec26806..bd3e166b 100644 --- a/src/test/java/com/teragrep/cfe_39/HdfsTest.java +++ b/src/test/java/com/teragrep/cfe_39/HdfsTest.java @@ -47,16 +47,8 @@ import com.google.gson.JsonObject; import com.google.gson.JsonParser; -import com.teragrep.cfe_39.avro.SyslogRecord; import com.teragrep.cfe_39.consumers.kafka.HDFSWrite; -import com.teragrep.cfe_39.consumers.kafka.HdfsDataIngestion; -import com.teragrep.cfe_39.consumers.kafka.RecordOffset; -import org.apache.avro.file.DataFileReader; -import org.apache.avro.file.DataFileStream; -import org.apache.avro.io.DatumReader; -import org.apache.avro.specific.SpecificDatumReader; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.Path; @@ -69,9 +61,11 @@ import java.io.File; import java.net.URI; import java.nio.file.Files; +import java.nio.file.Paths; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +// Tests the functionality of the HDFSWrite.java. public class HdfsTest { private static final Logger LOGGER = LoggerFactory.getLogger(HdfsTest.class); @@ -79,22 +73,13 @@ public class HdfsTest { private static MiniDFSCluster hdfsCluster; private static File baseDir; private static Config config; + private FileSystem fs; - // Generate AVRO-files for testing the HDFS writes. - @Disabled - @BeforeAll - public static void generateTestData() { + // Start minicluster and initialize config. + @BeforeEach + public void startMiniCluster() { assertDoesNotThrow(() -> { config = new Config(); - startMiniCluster(); - config.setMaximumFileSize(3000); // 10 loops (140 records) are in use at the moment, and that is sized at 36,102 bytes. - HdfsDataIngestion hdfsDataIngestion = new HdfsDataIngestion(config); - hdfsDataIngestion.run(); - }); - } - - public static void startMiniCluster() { - assertDoesNotThrow(() -> { // Create a HDFS miniCluster baseDir = Files.createTempDirectory("test_hdfs").toFile().getAbsoluteFile(); Configuration conf = new Configuration(); @@ -102,275 +87,134 @@ public static void startMiniCluster() { MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf); hdfsCluster = builder.build(); String hdfsURI = "hdfs://localhost:" + hdfsCluster.getNameNodePort() + "/"; - LOGGER.debug("hdfsURI: " + hdfsURI); config.setHdfsuri(hdfsURI); DistributedFileSystem fileSystem = hdfsCluster.getFileSystem(); - }); - } - // Delete the generated AVRO-files. - @Disabled - @AfterAll - public static void deleteTestData() { - Path queueDirectory = new Path(config.getQueueDirectory()); // Paths.get(config.getQueueDirectory()); - for (int j = 0; j <= 9; j++) { - for (int i = 1; i <= 2; i++) { - File syslogFile = new File(queueDirectory.toUri() + File.separator + "testConsumerTopic" + j + "." + i); - assertDoesNotThrow(() -> { - boolean result = Files.deleteIfExists(syslogFile.toPath()); - }); + // ====== Init HDFS File System Object + Configuration fsConf = new Configuration(); + // Set FileSystem URI + fsConf.set("fs.defaultFS", hdfsURI); + // Because of Maven + fsConf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); + fsConf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); + // Set HADOOP user + System.setProperty("HADOOP_USER_NAME", "hdfs"); + System.setProperty("hadoop.home.dir", "/"); + String path = config.getHdfsPath() + "/" + "testConsumerTopic"; // "hdfs:///opt/teragrep/cfe_39/srv/testConsumerTopic" + fs = FileSystem.get(URI.create(hdfsURI), fsConf); + //==== Create directory if not exists + Path workingDir = fs.getWorkingDirectory(); + // Sets the directory where the data should be stored, if the directory doesn't exist then it's created. + Path newDirectoryPath = new Path(path); + if (!fs.exists(newDirectoryPath)) { + // Create new Directory + fs.mkdirs(newDirectoryPath); + LOGGER.debug("Path {} created.", path); } - } - closeMiniCluster(); + }); } - public static void closeMiniCluster() { - // Teardown HDFS miniCluster + // Teardown the minicluster + @AfterEach + public void teardownMiniCluster() { + assertDoesNotThrow(() -> { + fs.close(); + }); hdfsCluster.shutdown(); FileUtil.fullyDelete(baseDir); } - @Disabled @Test public void hdfsWriteTest() { - startMiniCluster(); + // Tests HDFSWrite.java functionality by committing pre-generated AVRO-files to HDFS and assert if it worked as expected. assertDoesNotThrow(() -> { - Path queueDirectory = new Path(config.getQueueDirectory()); - for (int j = 0; j <= 9; j++) { - for (int i = 1; i <= 2; i++) { - File syslogFile = new File( - queueDirectory.toUri() + File.separator + "testConsumerTopic" + j + "." + i - ); - - // generate lastObject from the last record in the file in this test - DatumReader userDatumReader = new SpecificDatumReader<>(SyslogRecord.class); - SyslogRecord lastRecord = null; - try ( - DataFileReader dataFileReader = new DataFileReader<>( - syslogFile, - userDatumReader - ) - ) { - while (dataFileReader.hasNext()) { - lastRecord = dataFileReader.next(lastRecord); - } - } - - assert lastRecord != null; - RecordOffset lastObject = new RecordOffset( - "testConsumerTopic", - Integer.parseInt(lastRecord.getPartition().toString()), - lastRecord.getOffset(), - null - ); // Fetch input parameters from the lastRecord SyslogRecord-object. - JsonObject lastObjectJo = JsonParser.parseString(lastObject.offsetToJSON()).getAsJsonObject(); - if (LOGGER.isDebugEnabled()) { - LOGGER - .debug( - "\n" + "Last record in the " + syslogFile.getName() + " file:" + "\ntopic: " - + lastObjectJo.get("topic").getAsString() + "\npartition: " + lastObjectJo.get("partition").getAsString() + "\noffset: " + lastObjectJo.get("offset").getAsString() - ); - } - HDFSWrite writer = new HDFSWrite(config, lastObject, lastObjectJo); - writer.commit(syslogFile, -1L); // commits the final AVRO-file to HDFS. - Thread.sleep(1000); - hdfsReadCheck( - "testConsumerTopic", Integer.parseInt(lastRecord.getPartition().toString()), lastRecord.getOffset() - ); - - } + Assertions + .assertEquals(0, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); + + // writer.commit will delete the file that is given as an input argument. Copy the mock files to another directory so the deletion can be asserted properly too. + String pathname = System.getProperty("user.dir") + "/src/test/java/com/teragrep/cfe_39/mockHdfsFiles/0.9"; + java.nio.file.Path sourceFile = Paths.get(pathname); + java.nio.file.Path targetDir = Paths.get(config.getQueueDirectory()); + java.nio.file.Path targetFile = targetDir.resolve(sourceFile.getFileName()); + Assertions.assertFalse(targetFile.toFile().exists()); + Files.copy(sourceFile, targetFile); + Assertions.assertTrue(targetFile.toFile().exists()); + File avroFile = new File(targetFile.toUri()); + JsonObject recordOffsetJo = JsonParser + .parseString("{\"topic\":\"testConsumerTopic\", \"partition\":\"0\", \"offset\":\"9\"}") + .getAsJsonObject(); + try (HDFSWrite writer = new HDFSWrite(config, recordOffsetJo)) { + writer.commit(avroFile); // commits avroFile to HDFS and deletes avroFile afterward. + } + Assertions.assertFalse(targetFile.toFile().exists()); + Assertions + .assertEquals(1, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); + + pathname = System.getProperty("user.dir") + "/src/test/java/com/teragrep/cfe_39/mockHdfsFiles/0.13"; + sourceFile = Paths.get(pathname); + targetDir = Paths.get(config.getQueueDirectory()); + targetFile = targetDir.resolve(sourceFile.getFileName()); + Files.copy(sourceFile, targetFile); + Assertions.assertTrue(targetFile.toFile().exists()); + avroFile = new File(config.getQueueDirectory() + "/0.13"); + recordOffsetJo = JsonParser + .parseString("{\"topic\":\"testConsumerTopic\", \"partition\":\"0\", \"offset\":\"13\"}") + .getAsJsonObject(); + try (HDFSWrite writer = new HDFSWrite(config, recordOffsetJo)) { + writer.commit(avroFile); // commits avroFile to HDFS and deletes avroFile afterward. } - closeMiniCluster(); + Assertions.assertFalse(targetFile.toFile().exists()); + Assertions + .assertEquals(2, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.13"))); }); } - public void hdfsReadCheck(String testConsumerTopic, int partition, long offset) { - - // Check that the files were properly written to HDFS with a read test. - String hdfsuri = config.getHdfsuri(); - - String path = config.getHdfsPath() + "/" + testConsumerTopic; - String fileName = partition + "." + offset; - // ====== Init HDFS File System Object - Configuration conf = new Configuration(); - // Set FileSystem URI - conf.set("fs.defaultFS", hdfsuri); - // Because of Maven - conf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); - conf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); - // Set HADOOP user - System.setProperty("HADOOP_USER_NAME", "hdfs"); - System.setProperty("hadoop.home.dir", "/"); + @Test + public void hdfsWriteExceptionTest() { + // File already exists exception test, commits the same file twice to trigger the exception. assertDoesNotThrow(() -> { - //Get the filesystem - HDFS - FileSystem fs = FileSystem.get(URI.create(hdfsuri), conf); - - //==== Create directory if not exists - Path workingDir = fs.getWorkingDirectory(); - Path newDirectoryPath = new Path(path); - if (!fs.exists(newDirectoryPath)) { - // Create new Directory - fs.mkdirs(newDirectoryPath); - // logger.info("Path "+path+" created."); - } - - /*==== Read files - Create a path*/ - Path hdfsreadpath = new Path(newDirectoryPath + "/" + fileName); // The path should be the same that was used in writing the file to HDFS. - //Init input stream - FSDataInputStream inputStream = fs.open(hdfsreadpath); - //The data is in AVRO-format, so it can't be read as a string. - DataFileStream reader = new DataFileStream<>( - inputStream, - new SpecificDatumReader<>(SyslogRecord.class) - ); - SyslogRecord record = null; - int looper; - if (offset == 8) { - looper = 0; + Assertions + .assertEquals(0, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); + + // writer.commit will delete the source file that is given as an input argument. Copy the mock file to another directory so the deletion of the source file can be asserted properly. + String pathname = System.getProperty("user.dir") + "/src/test/java/com/teragrep/cfe_39/mockHdfsFiles/0.9"; + java.nio.file.Path sourceFile = Paths.get(pathname); + java.nio.file.Path targetDir = Paths.get(config.getQueueDirectory()); + java.nio.file.Path targetFile = targetDir.resolve(sourceFile.getFileName()); + Assertions.assertFalse(targetFile.toFile().exists()); + Files.copy(sourceFile, targetFile); + + Assertions.assertTrue(targetFile.toFile().exists()); + File avroFile = new File(targetFile.toUri()); + JsonObject recordOffsetJo = JsonParser + .parseString("{\"topic\":\"testConsumerTopic\", \"partition\":\"0\", \"offset\":\"9\"}") + .getAsJsonObject(); + try (HDFSWrite writer = new HDFSWrite(config, recordOffsetJo)) { + writer.commit(avroFile); // commits avroFile to HDFS and deletes avroFile afterward. } - else if (offset == 13) { - looper = 9; - } - else { - looper = 0; - Assertions.fail("The offset of the last record is not 8 or 13, which means a failed test."); - } - while (reader.hasNext()) { - record = reader.next(record); - if (LOGGER.isDebugEnabled()) { - LOGGER.debug(record.toString()); - } - // Assert records here like it is done in KafkaConsumerTest.avroReader(). - if (looper <= 0) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872090804000, \"message\": \"[WARN] 2022-04-25 07:34:50,804 com.teragrep.jla_02.Log4j Log - Log4j warn says hi!\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partition + "\", \"offset\": 0, \"origin\": \"jla-02.default\"}", - record.toString() - ); - looper++; - } - else if (looper == 1) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872090806000, \"message\": \"[ERROR] 2022-04-25 07:34:50,806 com.teragrep.jla_02.Log4j Log - Log4j error says hi!\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partition + "\", \"offset\": 1, \"origin\": \"jla-02.default\"}", - record.toString() - ); - looper++; - } - else if (looper == 2) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872090822000, \"message\": \"470647 [Thread-3] INFO com.teragrep.jla_02.Logback Daily - Logback-daily says hi.\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partition + "\", \"offset\": 2, \"origin\": \"jla-02\"}", - record.toString() - ); - looper++; - } - else if (looper == 3) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872090822000, \"message\": \"470646 [Thread-3] INFO com.teragrep.jla_02.Logback Audit - Logback-audit says hi.\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partition + "\", \"offset\": 3, \"origin\": \"jla-02\"}", - record.toString() - ); - looper++; - } - else if (looper == 4) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872090822000, \"message\": \"470647 [Thread-3] INFO com.teragrep.jla_02.Logback Metric - Logback-metric says hi.\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partition + "\", \"offset\": 4, \"origin\": \"jla-02\"}", - record.toString() - ); - looper++; - } - else if (looper == 5) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872092238000, \"message\": \"25.04.2022 07:34:52.238 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info audit says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partition + "\", \"offset\": 5, \"origin\": \"jla-02.default\"}", - record.toString() - ); - looper++; - } - else if (looper == 6) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872092239000, \"message\": \"25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info daily says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partition + "\", \"offset\": 6, \"origin\": \"jla-02.default\"}", - record.toString() - ); - looper++; - } - else if (looper == 7) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872092239000, \"message\": \"25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info metric says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partition + "\", \"offset\": 7, \"origin\": \"jla-02.default\"}", - record.toString() - ); - looper++; - } - else if (looper == 8) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872092240000, \"message\": \"25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn audit says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partition + "\", \"offset\": 8, \"origin\": \"jla-02.default\"}", - record.toString() - ); - looper++; - } - else if (looper == 9) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872092240000, \"message\": \"25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn daily says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partition + "\", \"offset\": 9, \"origin\": \"jla-02.default\"}", - record.toString() - ); - looper++; - } - else if (looper == 10) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872092241000, \"message\": \"25.04.2022 07:34:52.241 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn metric says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partition + "\", \"offset\": 10, \"origin\": \"jla-02.default\"}", - record.toString() - ); - looper++; - } - else if (looper == 11) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872092241000, \"message\": \"25.04.2022 07:34:52.241 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error audit says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partition + "\", \"offset\": 11, \"origin\": \"jla-02.default\"}", - record.toString() - ); - looper++; - } - else if (looper == 12) { - Assertions - .assertEquals( - "{\"timestamp\": 1650872092242000, \"message\": \"25.04.2022 07:34:52.242 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error daily says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partition + "\", \"offset\": 12, \"origin\": \"jla-02.default\"}", - record.toString() - ); - looper++; - } - else { - Assertions - .assertEquals( - "{\"timestamp\": 1650872092243000, \"message\": \"25.04.2022 07:34:52.243 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error metric says hi!]\", \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partition + "\", \"offset\": 13, \"origin\": \"jla-02.default\"}", - record.toString() - ); - looper = 0; - } - } - inputStream.close(); - fs.close(); + Assertions.assertFalse(targetFile.toFile().exists()); + Assertions + .assertEquals(1, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); + + Files.copy(sourceFile, targetFile); + Assertions.assertTrue(targetFile.toFile().exists()); + avroFile = new File(config.getQueueDirectory() + "/0.9"); + recordOffsetJo = JsonParser + .parseString("{\"topic\":\"testConsumerTopic\", \"partition\":\"0\", \"offset\":\"9\"}") + .getAsJsonObject(); + HDFSWrite writer = new HDFSWrite(config, recordOffsetJo); + File finalAvroFile = avroFile; + Exception e = Assertions.assertThrows(Exception.class, () -> writer.commit(finalAvroFile)); + Assertions.assertEquals("File 0.9 already exists", e.getMessage()); + writer.close(); + Assertions.assertFalse(targetFile.toFile().exists()); + Assertions + .assertEquals(1, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); }); } } From 728a6dea2faecc915d6319f5e5d82f6016c3e583 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Mon, 17 Jun 2024 16:20:27 +0300 Subject: [PATCH 095/146] Improved test assertion setups. --- .../com/teragrep/cfe_39/CombinedFullTest.java | 29 ++++++++++--------- .../java/com/teragrep/cfe_39/ConfigTest.java | 4 +++ .../java/com/teragrep/cfe_39/HdfsTest.java | 16 ++-------- .../java/com/teragrep/cfe_39/PruningTest.java | 3 +- 4 files changed, 23 insertions(+), 29 deletions(-) diff --git a/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java b/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java index ba055f60..d5be19bd 100644 --- a/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java +++ b/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java @@ -100,17 +100,7 @@ public void startMiniCluster() { // Set HADOOP user System.setProperty("HADOOP_USER_NAME", "hdfs"); System.setProperty("hadoop.home.dir", "/"); - String path = config.getHdfsPath() + "/" + "testConsumerTopic"; // "hdfs:///opt/teragrep/cfe_39/srv/testConsumerTopic" fs = FileSystem.get(URI.create(hdfsURI), fsConf); - //==== Create directory if not exists - Path workingDir = fs.getWorkingDirectory(); - // Sets the directory where the data should be stored, if the directory doesn't exist then it's created. - Path newDirectoryPath = new Path(path); - if (!fs.exists(newDirectoryPath)) { - // Create new Directory - fs.mkdirs(newDirectoryPath); - LOGGER.debug("Path {} created.", path); - } }); } @@ -130,12 +120,14 @@ public void ingestionTest0Files() { assertDoesNotThrow(() -> { Assertions.assertTrue(config.getPruneOffset() >= 300000L); // Fails the test if the config is not correct. config.setMaximumFileSize(30000); // This parameter defines the amount of records that can fit inside a single AVRO-file. + Assertions.assertFalse(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic"))); HdfsDataIngestion hdfsDataIngestion = new HdfsDataIngestion(config); Thread.sleep(10000); hdfsDataIngestion.run(); - // Assert that the kafka records were ingested correctly and the database holds the correct 140 records. }); + // Assert that the kafka records were ingested correctly and the database holds the correct 140 records. + // Check that the files were properly written to HDFS. String hdfsuri = config.getHdfsuri(); @@ -152,7 +144,7 @@ public void ingestionTest0Files() { System.setProperty("hadoop.home.dir", "/"); //Get the filesystem - HDFS assertDoesNotThrow(() -> { - FileSystem fs = FileSystem.get(URI.create(hdfsuri), conf); + fs = FileSystem.get(URI.create(hdfsuri), conf); Path workingDir = fs.getWorkingDirectory(); Path newDirectoryPath = new Path(path); @@ -339,9 +331,12 @@ record = reader.next(record); @Test public void ingestionTest2OldFiles() { - /* 14 records are inserted to HDFS database before starting ingestion, with 124/140 records in mock kafka consumer ready for ingestion. + /* 14 records are inserted to HDFS database before starting ingestion, with 126/140 records in mock kafka consumer ready for ingestion. Partitions through 1 to 9 will have only a single file, partition 0 will have 2 files (0.9 and 0.13). partition 0 files are pre-made and inserted to the HDFS database with old timestamps that will mark them for pruning when ingestion is started.*/ + assertDoesNotThrow(() -> { + Assertions.assertFalse(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic"))); + }); insertMockFiles(157784760000L, 157784760000L); // Insert 2 mock files (0.9 and 0.13) with old timestamps so pruning should trigger on them. assertDoesNotThrow(() -> { @@ -395,6 +390,9 @@ public void ingestionTest1Old1NewFile() { /* 14 records are inserted to HDFS database before starting ingestion, with 124/140 records in mock kafka consumer ready for ingestion. Partitions through 1 to 9 will have only a single file, partition 0 will have 2 files (0.9 and 0.13). partition 0 files are pre-made and inserted to the HDFS database with old timestamp for file 0.9 and new for 0.13.*/ + assertDoesNotThrow(() -> { + Assertions.assertFalse(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic"))); + }); insertMockFiles(157784760000L, -1); // Insert 2 mock files (0.9 and 0.13) with old timestamp on 0.9 and new timestamp on 0.13. assertDoesNotThrow(() -> { @@ -445,6 +443,9 @@ public void ingestionTest1Old1NewFile() { @Test public void ingestionTest2NewFiles() { // 14 records are inserted to HDFS database before starting ingestion, with 124/140 records in mock kafka consumer ready for ingestion. Partitions through 1 to 9 will have only a single file, partition 0 will have 2 files (0.9 and 0.13). + assertDoesNotThrow(() -> { + Assertions.assertFalse(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic"))); + }); insertMockFiles(-1, -1); // Insert 2 mock files (0.9 and 0.13) with new timestamps so pruning should not trigger on them. assertDoesNotThrow(() -> { @@ -506,7 +507,7 @@ public void ingestionTest2NewFiles() { System.setProperty("hadoop.home.dir", "/"); //Get the filesystem - HDFS assertDoesNotThrow(() -> { - FileSystem fs = FileSystem.get(URI.create(hdfsuri), conf); + fs = FileSystem.get(URI.create(hdfsuri), conf); Path workingDir = fs.getWorkingDirectory(); Path newDirectoryPath = new Path(path); diff --git a/src/test/java/com/teragrep/cfe_39/ConfigTest.java b/src/test/java/com/teragrep/cfe_39/ConfigTest.java index 8a8a0ea5..2f3607d1 100644 --- a/src/test/java/com/teragrep/cfe_39/ConfigTest.java +++ b/src/test/java/com/teragrep/cfe_39/ConfigTest.java @@ -45,6 +45,8 @@ */ package com.teragrep.cfe_39; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -57,6 +59,7 @@ public class ConfigTest { private static final Logger LOGGER = LoggerFactory.getLogger(ConfigTest.class); + @Disabled(value = "This code is WIP, leaving the test implementation to deployment step.") @Test public void configTest() { assertDoesNotThrow(() -> { @@ -65,6 +68,7 @@ public void configTest() { // Test extracting useMockKafkaConsumer value from config. boolean useMockKafkaConsumer = Boolean .parseBoolean(readerKafkaProperties.getProperty("useMockKafkaConsumer", "false")); + Assertions.assertTrue(useMockKafkaConsumer); LOGGER.debug("useMockKafkaConsumer: " + useMockKafkaConsumer); }); } diff --git a/src/test/java/com/teragrep/cfe_39/HdfsTest.java b/src/test/java/com/teragrep/cfe_39/HdfsTest.java index bd3e166b..96dad864 100644 --- a/src/test/java/com/teragrep/cfe_39/HdfsTest.java +++ b/src/test/java/com/teragrep/cfe_39/HdfsTest.java @@ -100,17 +100,7 @@ public void startMiniCluster() { // Set HADOOP user System.setProperty("HADOOP_USER_NAME", "hdfs"); System.setProperty("hadoop.home.dir", "/"); - String path = config.getHdfsPath() + "/" + "testConsumerTopic"; // "hdfs:///opt/teragrep/cfe_39/srv/testConsumerTopic" fs = FileSystem.get(URI.create(hdfsURI), fsConf); - //==== Create directory if not exists - Path workingDir = fs.getWorkingDirectory(); - // Sets the directory where the data should be stored, if the directory doesn't exist then it's created. - Path newDirectoryPath = new Path(path); - if (!fs.exists(newDirectoryPath)) { - // Create new Directory - fs.mkdirs(newDirectoryPath); - LOGGER.debug("Path {} created.", path); - } }); } @@ -128,8 +118,7 @@ public void teardownMiniCluster() { public void hdfsWriteTest() { // Tests HDFSWrite.java functionality by committing pre-generated AVRO-files to HDFS and assert if it worked as expected. assertDoesNotThrow(() -> { - Assertions - .assertEquals(0, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); + Assertions.assertFalse(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic"))); // writer.commit will delete the file that is given as an input argument. Copy the mock files to another directory so the deletion can be asserted properly too. String pathname = System.getProperty("user.dir") + "/src/test/java/com/teragrep/cfe_39/mockHdfsFiles/0.9"; @@ -176,8 +165,7 @@ public void hdfsWriteTest() { public void hdfsWriteExceptionTest() { // File already exists exception test, commits the same file twice to trigger the exception. assertDoesNotThrow(() -> { - Assertions - .assertEquals(0, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); + Assertions.assertFalse(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic"))); // writer.commit will delete the source file that is given as an input argument. Copy the mock file to another directory so the deletion of the source file can be asserted properly. String pathname = System.getProperty("user.dir") + "/src/test/java/com/teragrep/cfe_39/mockHdfsFiles/0.9"; diff --git a/src/test/java/com/teragrep/cfe_39/PruningTest.java b/src/test/java/com/teragrep/cfe_39/PruningTest.java index 49c0472d..37405276 100644 --- a/src/test/java/com/teragrep/cfe_39/PruningTest.java +++ b/src/test/java/com/teragrep/cfe_39/PruningTest.java @@ -122,7 +122,8 @@ public void twoNewFiles() { insertMockFiles(-1, -1); // Insert 2 mock files with new timestamps so pruning should not trigger on them. assertDoesNotThrow(() -> { - Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic"))); + Assertions + .assertEquals(2, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.13"))); HDFSPrune hdfsPrune = new HDFSPrune(config, "testConsumerTopic"); From dee838a1920ce47e53594859188692fde46535f7 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Tue, 18 Jun 2024 10:37:26 +0300 Subject: [PATCH 096/146] Separated tests in CombinedFullTest.java to Ingestion0FilesTest.java, Ingestion2OldFilesTest.java, Ingestion1Old1NewFileTest.java and Ingestion2NewFilesTest.java. --- .../teragrep/cfe_39/Ingestion0FilesTest.java | 329 ++++++++++++++++ .../cfe_39/Ingestion1Old1NewFileTest.java | 217 +++++++++++ ...lTest.java => Ingestion2NewFilesTest.java} | 351 +----------------- .../cfe_39/Ingestion2OldFilesTest.java | 218 +++++++++++ 4 files changed, 778 insertions(+), 337 deletions(-) create mode 100644 src/test/java/com/teragrep/cfe_39/Ingestion0FilesTest.java create mode 100644 src/test/java/com/teragrep/cfe_39/Ingestion1Old1NewFileTest.java rename src/test/java/com/teragrep/cfe_39/{CombinedFullTest.java => Ingestion2NewFilesTest.java} (61%) create mode 100644 src/test/java/com/teragrep/cfe_39/Ingestion2OldFilesTest.java diff --git a/src/test/java/com/teragrep/cfe_39/Ingestion0FilesTest.java b/src/test/java/com/teragrep/cfe_39/Ingestion0FilesTest.java new file mode 100644 index 00000000..be528bd0 --- /dev/null +++ b/src/test/java/com/teragrep/cfe_39/Ingestion0FilesTest.java @@ -0,0 +1,329 @@ +/* + * HDFS Data Ingestion for PTH_06 use CFE-39 + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. + */ +package com.teragrep.cfe_39; + +import com.teragrep.cfe_39.avro.SyslogRecord; +import com.teragrep.cfe_39.consumers.kafka.HdfsDataIngestion; +import org.apache.avro.file.DataFileStream; +import org.apache.avro.specific.SpecificDatumReader; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.*; +import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.junit.jupiter.api.*; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.hadoop.fs.Path; + +import java.io.File; +import java.net.URI; +import java.nio.file.Files; +import java.util.*; + +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; + +public class Ingestion0FilesTest { + + private static final Logger LOGGER = LoggerFactory.getLogger(Ingestion0FilesTest.class); + private static MiniDFSCluster hdfsCluster; + private static File baseDir; + private static Config config; + private FileSystem fs; + + // Prepares known state for testing. + @BeforeEach + public void startMiniCluster() { + assertDoesNotThrow(() -> { + config = new Config(); + // Create a HDFS miniCluster + baseDir = Files.createTempDirectory("test_hdfs").toFile().getAbsoluteFile(); + Configuration conf = new Configuration(); + conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, baseDir.getAbsolutePath()); + MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf); + hdfsCluster = builder.build(); + String hdfsURI = "hdfs://localhost:" + hdfsCluster.getNameNodePort() + "/"; + config.setHdfsuri(hdfsURI); + DistributedFileSystem fileSystem = hdfsCluster.getFileSystem(); + + // ====== Init HDFS File System Object + Configuration fsConf = new Configuration(); + // Set FileSystem URI + fsConf.set("fs.defaultFS", hdfsURI); + // Because of Maven + fsConf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); + fsConf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); + // Set HADOOP user + System.setProperty("HADOOP_USER_NAME", "hdfs"); + System.setProperty("hadoop.home.dir", "/"); + fs = FileSystem.get(URI.create(hdfsURI), fsConf); + }); + } + + // Teardown the minicluster + @AfterEach + public void teardownMiniCluster() { + assertDoesNotThrow(() -> { + fs.close(); + }); + hdfsCluster.shutdown(); + FileUtil.fullyDelete(baseDir); + } + + @Test + public void ingestion0FilesTest() { + // Empty HDFS database, 140 records in mock kafka consumer ready for ingestion. All 14 records for each 10 topic partitions are stored in a single avro-file per partition. + assertDoesNotThrow(() -> { + Assertions.assertTrue(config.getPruneOffset() >= 300000L); // Fails the test if the config is not correct. + config.setMaximumFileSize(30000); // This parameter defines the amount of records that can fit inside a single AVRO-file. + Assertions.assertFalse(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic"))); + HdfsDataIngestion hdfsDataIngestion = new HdfsDataIngestion(config); + Thread.sleep(10000); + hdfsDataIngestion.run(); + }); + + // Assert that the kafka records were ingested correctly and the database holds the correct 140 records. + + // Check that the files were properly written to HDFS. + String hdfsuri = config.getHdfsuri(); + + String path = config.getHdfsPath() + "/" + "testConsumerTopic"; + // ====== Init HDFS File System Object + Configuration conf = new Configuration(); + // Set FileSystem URI + conf.set("fs.defaultFS", hdfsuri); + // Because of Maven + conf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); + conf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); + // Set HADOOP user + System.setProperty("HADOOP_USER_NAME", "hdfs"); + System.setProperty("hadoop.home.dir", "/"); + //Get the filesystem - HDFS + assertDoesNotThrow(() -> { + fs = FileSystem.get(URI.create(hdfsuri), conf); + + Path workingDir = fs.getWorkingDirectory(); + Path newDirectoryPath = new Path(path); + Assertions.assertTrue(fs.exists(newDirectoryPath)); + + /* This is the HDFS write path for the files: + Path hdfswritepath = new Path(newDirectoryPath + "/" + fileName); where newDirectoryPath is config.getHdfsPath() + "/" + lastObject.topic; and filename is lastObject.partition+"."+lastObject.offset; + + Create the list of files to read from HDFS. Test setup is created so each of the 0-9 partitions will have 1 file with offset of 13.*/ + List filenameList = new ArrayList<>(); + for (int i = 0; i <= 9; i++) { + filenameList.add(i + "." + 13); + } + FileStatus[] fileStatuses = fs.listStatus(newDirectoryPath); + Assertions.assertEquals(filenameList.size(), fileStatuses.length); + for (FileStatus fileStatus : fileStatuses) { + Assertions.assertTrue(filenameList.contains(fileStatus.getPath().getName())); + } + LOGGER.debug("All expected files present in HDFS."); + + int partitionCounter = 0; + for (String fileName : filenameList) { + //==== Read files + LOGGER.info("Read file into hdfs"); + //Create a path + Path hdfsreadpath = new Path(newDirectoryPath + "/" + fileName); // The path should be the same that was used in writing the file to HDFS. + //Init input stream + FSDataInputStream inputStream = fs.open(hdfsreadpath); + //The data is in AVRO-format, so it can't be read as a string. + DataFileStream reader = new DataFileStream<>( + inputStream, + new SpecificDatumReader<>(SyslogRecord.class) + ); + SyslogRecord record = null; + LOGGER.info("\nReading records from file {}:", hdfsreadpath); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + Assertions + .assertEquals( + "{\"timestamp\": 1650872090804000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 0, \"origin\": \"jla-02.default\", \"payload\": \"[WARN] 2022-04-25 07:34:50,804 com.teragrep.jla_02.Log4j Log - Log4j warn says hi!\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + Assertions + .assertEquals( + "{\"timestamp\": 1650872090806000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 1, \"origin\": \"jla-02.default\", \"payload\": \"[ERROR] 2022-04-25 07:34:50,806 com.teragrep.jla_02.Log4j Log - Log4j error says hi!\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + Assertions + .assertEquals( + "{\"timestamp\": 1650872090822000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 2, \"origin\": \"jla-02\", \"payload\": \"470647 [Thread-3] INFO com.teragrep.jla_02.Logback Daily - Logback-daily says hi.\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + Assertions + .assertEquals( + "{\"timestamp\": 1650872090822000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 3, \"origin\": \"jla-02\", \"payload\": \"470646 [Thread-3] INFO com.teragrep.jla_02.Logback Audit - Logback-audit says hi.\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + Assertions + .assertEquals( + "{\"timestamp\": 1650872090822000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 4, \"origin\": \"jla-02\", \"payload\": \"470647 [Thread-3] INFO com.teragrep.jla_02.Logback Metric - Logback-metric says hi.\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + Assertions + .assertEquals( + "{\"timestamp\": 1650872092238000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 5, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.238 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info audit says hi!]\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + Assertions + .assertEquals( + "{\"timestamp\": 1650872092239000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 6, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info daily says hi!]\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + Assertions + .assertEquals( + "{\"timestamp\": 1650872092239000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 7, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info metric says hi!]\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + Assertions + .assertEquals( + "{\"timestamp\": 1650872092240000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 8, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn audit says hi!]\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + Assertions + .assertEquals( + "{\"timestamp\": 1650872092240000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 9, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn daily says hi!]\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + Assertions + .assertEquals( + "{\"timestamp\": 1650872092241000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 10, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.241 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn metric says hi!]\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + Assertions + .assertEquals( + "{\"timestamp\": 1650872092241000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 11, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.241 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error audit says hi!]\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + Assertions + .assertEquals( + "{\"timestamp\": 1650872092242000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 12, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.242 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error daily says hi!]\"}", + record.toString() + ); + + Assertions.assertTrue(reader.hasNext()); + record = reader.next(record); + Assertions + .assertEquals( + "{\"timestamp\": 1650872092243000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" + + partitionCounter + + "\", \"offset\": 13, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.243 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error metric says hi!]\"}", + record.toString() + ); + Assertions.assertFalse(reader.hasNext()); + LOGGER.info("Partition {} passed assertions.", partitionCounter); + partitionCounter++; + inputStream.close(); + } + Assertions.assertEquals(10, partitionCounter); + }); + } +} diff --git a/src/test/java/com/teragrep/cfe_39/Ingestion1Old1NewFileTest.java b/src/test/java/com/teragrep/cfe_39/Ingestion1Old1NewFileTest.java new file mode 100644 index 00000000..0d5f0fa9 --- /dev/null +++ b/src/test/java/com/teragrep/cfe_39/Ingestion1Old1NewFileTest.java @@ -0,0 +1,217 @@ +/* + * HDFS Data Ingestion for PTH_06 use CFE-39 + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. + */ +package com.teragrep.cfe_39; + +import com.teragrep.cfe_39.consumers.kafka.HdfsDataIngestion; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FileUtil; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.net.URI; +import java.nio.file.Files; +import java.util.Objects; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; + +public class Ingestion1Old1NewFileTest { + + private static final Logger LOGGER = LoggerFactory.getLogger(Ingestion1Old1NewFileTest.class); + private static MiniDFSCluster hdfsCluster; + private static File baseDir; + private static Config config; + private FileSystem fs; + + // Prepares known state for testing. + @BeforeEach + public void startMiniCluster() { + assertDoesNotThrow(() -> { + config = new Config(); + // Create a HDFS miniCluster + baseDir = Files.createTempDirectory("test_hdfs").toFile().getAbsoluteFile(); + Configuration conf = new Configuration(); + conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, baseDir.getAbsolutePath()); + MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf); + hdfsCluster = builder.build(); + String hdfsURI = "hdfs://localhost:" + hdfsCluster.getNameNodePort() + "/"; + config.setHdfsuri(hdfsURI); + DistributedFileSystem fileSystem = hdfsCluster.getFileSystem(); + + // ====== Init HDFS File System Object + Configuration fsConf = new Configuration(); + // Set FileSystem URI + fsConf.set("fs.defaultFS", hdfsURI); + // Because of Maven + fsConf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); + fsConf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); + // Set HADOOP user + System.setProperty("HADOOP_USER_NAME", "hdfs"); + System.setProperty("hadoop.home.dir", "/"); + fs = FileSystem.get(URI.create(hdfsURI), fsConf); + insertMockFiles(157784760000L, -1); // Insert 2 mock files (0.9 and 0.13) with old timestamp on 0.9 and new timestamp on 0.13. + }); + } + + // Teardown the minicluster + @AfterEach + public void teardownMiniCluster() { + assertDoesNotThrow(() -> { + fs.close(); + }); + hdfsCluster.shutdown(); + FileUtil.fullyDelete(baseDir); + } + + @Test + public void ingestion1Old1NewFileTest() { + /* 14 records are inserted to HDFS database before starting ingestion, with 124/140 records in mock kafka consumer ready for ingestion. + Partitions through 1 to 9 will have only a single file, partition 0 will have 2 files (0.9 and 0.13). + partition 0 files are pre-made and inserted to the HDFS database with old timestamp for file 0.9 and new for 0.13.*/ + + assertDoesNotThrow(() -> { + // Assert the known starting state. + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic"))); + Assertions + .assertEquals(2, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.13"))); + Assertions.assertTrue(config.getPruneOffset() >= 300000L); // Fails the test if the config is not correct. + Assertions.assertTrue((System.currentTimeMillis() - config.getPruneOffset()) > 157784760000L); + config.setMaximumFileSize(30000); + HdfsDataIngestion hdfsDataIngestion = new HdfsDataIngestion(config); + Thread.sleep(10000); + hdfsDataIngestion.run(); + + // hdfsDataIngestion.run(); has called fs.close() after finishing ingesting the records from kafka. Rebuild fs. + String hdfsURI = "hdfs://localhost:" + hdfsCluster.getNameNodePort() + "/"; + config.setHdfsuri(hdfsURI); + // ====== Init HDFS File System Object + Configuration fsConf = new Configuration(); + // Set FileSystem URI + fsConf.set("fs.defaultFS", hdfsURI); + // Because of Maven + fsConf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); + fsConf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); + // Set HADOOP user + System.setProperty("HADOOP_USER_NAME", "hdfs"); + System.setProperty("hadoop.home.dir", "/"); + fs = FileSystem.get(URI.create(hdfsURI), fsConf); + + // Assert that the kafka records were ingested and pruned correctly and the database holds only the expected 10 files. + Assertions + .assertEquals(10, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); + Assertions.assertFalse(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.13"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "1.13"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "2.13"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "3.13"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "4.13"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "5.13"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "6.13"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "7.13"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "8.13"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "9.13"))); + }); + } + + // Inserts pre-made avro-files to HDFS, which are normally generated during data ingestion from mock kafka consumer. + private void insertMockFiles(long fileTimestampA, long fileTimestampB) { + String path = config.getHdfsPath() + "/" + "testConsumerTopic"; // "hdfs:///opt/teragrep/cfe_39/srv/testConsumerTopic" + //Get the filesystem - HDFS + assertDoesNotThrow(() -> { + + // Sets the directory where the data should be stored, if the directory doesn't exist then it's created. + Path newDirectoryPath = new Path(path); + if (!fs.exists(newDirectoryPath)) { + // Create new Directory + fs.mkdirs(newDirectoryPath); + LOGGER.debug("Path {} created.", path); + } + + String dir = System.getProperty("user.dir") + "/src/test/java/com/teragrep/cfe_39/mockHdfsFiles"; + Set listOfFiles = Stream + .of(Objects.requireNonNull(new File(dir).listFiles())) + .filter(file -> !file.isDirectory()) + .map(File::getName) + .collect(Collectors.toSet()); + // Loop through all the avro files + for (String fileName : listOfFiles) { + String pathname = dir + "/" + fileName; + File avroFile = new File(pathname); + //==== Write file + LOGGER.debug("Begin Write file into hdfs"); + //Create a path + Path hdfswritepath = new Path(newDirectoryPath + "/" + avroFile.getName()); // filename should be set according to the requirements: 0.12345 where 0 is Kafka partition and 12345 is Kafka offset. + Assertions.assertFalse(fs.exists(hdfswritepath)); + Path readPath = new Path(avroFile.getPath()); + fs.copyFromLocalFile(readPath, hdfswritepath); + // Set fileTimestampA/fileTimestampB to something like 157784760000 to trigger pruning, -1 to not alter the timestamp. + if (Objects.equals(hdfswritepath.toString(), "hdfs:/opt/teragrep/cfe_39/srv/testConsumerTopic/0.9")) { + fs.setTimes(hdfswritepath, fileTimestampA, -1); + } + else if ( + Objects.equals(hdfswritepath.toString(), "hdfs:/opt/teragrep/cfe_39/srv/testConsumerTopic/0.13") + ) { + fs.setTimes(hdfswritepath, fileTimestampB, -1); + } + LOGGER.debug("End Write file into hdfs"); + LOGGER.debug("\nFile committed to HDFS, file writepath should be: {}\n", hdfswritepath); + } + }); + } +} diff --git a/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java b/src/test/java/com/teragrep/cfe_39/Ingestion2NewFilesTest.java similarity index 61% rename from src/test/java/com/teragrep/cfe_39/CombinedFullTest.java rename to src/test/java/com/teragrep/cfe_39/Ingestion2NewFilesTest.java index d5be19bd..f8eb2aee 100644 --- a/src/test/java/com/teragrep/cfe_39/CombinedFullTest.java +++ b/src/test/java/com/teragrep/cfe_39/Ingestion2NewFilesTest.java @@ -53,29 +53,34 @@ import org.apache.hadoop.fs.*; import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.MiniDFSCluster; -import org.junit.jupiter.api.*; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hadoop.fs.Path; import java.io.File; import java.net.URI; import java.nio.file.Files; -import java.util.*; +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; +import java.util.Set; import java.util.stream.Collectors; import java.util.stream.Stream; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; -public class CombinedFullTest { +public class Ingestion2NewFilesTest { - private static final Logger LOGGER = LoggerFactory.getLogger(CombinedFullTest.class); + private static final Logger LOGGER = LoggerFactory.getLogger(Ingestion2NewFilesTest.class); private static MiniDFSCluster hdfsCluster; private static File baseDir; private static Config config; private FileSystem fs; - // Start minicluster and initialize config. + // Prepares known state for testing. @BeforeEach public void startMiniCluster() { assertDoesNotThrow(() -> { @@ -101,6 +106,7 @@ public void startMiniCluster() { System.setProperty("HADOOP_USER_NAME", "hdfs"); System.setProperty("hadoop.home.dir", "/"); fs = FileSystem.get(URI.create(hdfsURI), fsConf); + insertMockFiles(-1, -1); // Insert 2 mock files (0.9 and 0.13) with new timestamps so pruning should not trigger on them. }); } @@ -115,340 +121,11 @@ public void teardownMiniCluster() { } @Test - public void ingestionTest0Files() { - // Empty HDFS database, 140 records in mock kafka consumer ready for ingestion. All 14 records for each 10 topic partitions are stored in a single avro-file per partition. - assertDoesNotThrow(() -> { - Assertions.assertTrue(config.getPruneOffset() >= 300000L); // Fails the test if the config is not correct. - config.setMaximumFileSize(30000); // This parameter defines the amount of records that can fit inside a single AVRO-file. - Assertions.assertFalse(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic"))); - HdfsDataIngestion hdfsDataIngestion = new HdfsDataIngestion(config); - Thread.sleep(10000); - hdfsDataIngestion.run(); - }); - - // Assert that the kafka records were ingested correctly and the database holds the correct 140 records. - - // Check that the files were properly written to HDFS. - String hdfsuri = config.getHdfsuri(); - - String path = config.getHdfsPath() + "/" + "testConsumerTopic"; - // ====== Init HDFS File System Object - Configuration conf = new Configuration(); - // Set FileSystem URI - conf.set("fs.defaultFS", hdfsuri); - // Because of Maven - conf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); - conf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); - // Set HADOOP user - System.setProperty("HADOOP_USER_NAME", "hdfs"); - System.setProperty("hadoop.home.dir", "/"); - //Get the filesystem - HDFS - assertDoesNotThrow(() -> { - fs = FileSystem.get(URI.create(hdfsuri), conf); - - Path workingDir = fs.getWorkingDirectory(); - Path newDirectoryPath = new Path(path); - Assertions.assertTrue(fs.exists(newDirectoryPath)); - - /* This is the HDFS write path for the files: - Path hdfswritepath = new Path(newDirectoryPath + "/" + fileName); where newDirectoryPath is config.getHdfsPath() + "/" + lastObject.topic; and filename is lastObject.partition+"."+lastObject.offset; - - Create the list of files to read from HDFS. Test setup is created so each of the 0-9 partitions will have 1 file with offset of 13.*/ - List filenameList = new ArrayList<>(); - for (int i = 0; i <= 9; i++) { - filenameList.add(i + "." + 13); - } - FileStatus[] fileStatuses = fs.listStatus(newDirectoryPath); - Assertions.assertEquals(filenameList.size(), fileStatuses.length); - for (FileStatus fileStatus : fileStatuses) { - Assertions.assertTrue(filenameList.contains(fileStatus.getPath().getName())); - } - LOGGER.debug("All expected files present in HDFS."); - - int partitionCounter = 0; - for (String fileName : filenameList) { - //==== Read files - LOGGER.info("Read file into hdfs"); - //Create a path - Path hdfsreadpath = new Path(newDirectoryPath + "/" + fileName); // The path should be the same that was used in writing the file to HDFS. - //Init input stream - FSDataInputStream inputStream = fs.open(hdfsreadpath); - //The data is in AVRO-format, so it can't be read as a string. - DataFileStream reader = new DataFileStream<>( - inputStream, - new SpecificDatumReader<>(SyslogRecord.class) - ); - SyslogRecord record = null; - LOGGER.info("\nReading records from file {}:", hdfsreadpath); - - Assertions.assertTrue(reader.hasNext()); - record = reader.next(record); - Assertions - .assertEquals( - "{\"timestamp\": 1650872090804000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 0, \"origin\": \"jla-02.default\", \"payload\": \"[WARN] 2022-04-25 07:34:50,804 com.teragrep.jla_02.Log4j Log - Log4j warn says hi!\"}", - record.toString() - ); - - Assertions.assertTrue(reader.hasNext()); - record = reader.next(record); - Assertions - .assertEquals( - "{\"timestamp\": 1650872090806000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 1, \"origin\": \"jla-02.default\", \"payload\": \"[ERROR] 2022-04-25 07:34:50,806 com.teragrep.jla_02.Log4j Log - Log4j error says hi!\"}", - record.toString() - ); - - Assertions.assertTrue(reader.hasNext()); - record = reader.next(record); - Assertions - .assertEquals( - "{\"timestamp\": 1650872090822000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 2, \"origin\": \"jla-02\", \"payload\": \"470647 [Thread-3] INFO com.teragrep.jla_02.Logback Daily - Logback-daily says hi.\"}", - record.toString() - ); - - Assertions.assertTrue(reader.hasNext()); - record = reader.next(record); - Assertions - .assertEquals( - "{\"timestamp\": 1650872090822000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 3, \"origin\": \"jla-02\", \"payload\": \"470646 [Thread-3] INFO com.teragrep.jla_02.Logback Audit - Logback-audit says hi.\"}", - record.toString() - ); - - Assertions.assertTrue(reader.hasNext()); - record = reader.next(record); - Assertions - .assertEquals( - "{\"timestamp\": 1650872090822000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 4, \"origin\": \"jla-02\", \"payload\": \"470647 [Thread-3] INFO com.teragrep.jla_02.Logback Metric - Logback-metric says hi.\"}", - record.toString() - ); - - Assertions.assertTrue(reader.hasNext()); - record = reader.next(record); - Assertions - .assertEquals( - "{\"timestamp\": 1650872092238000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 5, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.238 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info audit says hi!]\"}", - record.toString() - ); - - Assertions.assertTrue(reader.hasNext()); - record = reader.next(record); - Assertions - .assertEquals( - "{\"timestamp\": 1650872092239000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 6, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info daily says hi!]\"}", - record.toString() - ); - - Assertions.assertTrue(reader.hasNext()); - record = reader.next(record); - Assertions - .assertEquals( - "{\"timestamp\": 1650872092239000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 7, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.239 [INFO] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 info metric says hi!]\"}", - record.toString() - ); - - Assertions.assertTrue(reader.hasNext()); - record = reader.next(record); - Assertions - .assertEquals( - "{\"timestamp\": 1650872092240000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 8, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn audit says hi!]\"}", - record.toString() - ); - - Assertions.assertTrue(reader.hasNext()); - record = reader.next(record); - Assertions - .assertEquals( - "{\"timestamp\": 1650872092240000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 9, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.240 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn daily says hi!]\"}", - record.toString() - ); - - Assertions.assertTrue(reader.hasNext()); - record = reader.next(record); - Assertions - .assertEquals( - "{\"timestamp\": 1650872092241000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 10, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.241 [WARN] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 warn metric says hi!]\"}", - record.toString() - ); - - Assertions.assertTrue(reader.hasNext()); - record = reader.next(record); - Assertions - .assertEquals( - "{\"timestamp\": 1650872092241000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 11, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.241 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error audit says hi!]\"}", - record.toString() - ); - - Assertions.assertTrue(reader.hasNext()); - record = reader.next(record); - Assertions - .assertEquals( - "{\"timestamp\": 1650872092242000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 12, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.242 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error daily says hi!]\"}", - record.toString() - ); - - Assertions.assertTrue(reader.hasNext()); - record = reader.next(record); - Assertions - .assertEquals( - "{\"timestamp\": 1650872092243000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"" - + partitionCounter - + "\", \"offset\": 13, \"origin\": \"jla-02.default\", \"payload\": \"25.04.2022 07:34:52.243 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error metric says hi!]\"}", - record.toString() - ); - Assertions.assertFalse(reader.hasNext()); - LOGGER.info("Partition {} passed assertions.", partitionCounter); - partitionCounter++; - inputStream.close(); - } - Assertions.assertEquals(10, partitionCounter); - }); - } - - @Test - public void ingestionTest2OldFiles() { - /* 14 records are inserted to HDFS database before starting ingestion, with 126/140 records in mock kafka consumer ready for ingestion. - Partitions through 1 to 9 will have only a single file, partition 0 will have 2 files (0.9 and 0.13). - partition 0 files are pre-made and inserted to the HDFS database with old timestamps that will mark them for pruning when ingestion is started.*/ - assertDoesNotThrow(() -> { - Assertions.assertFalse(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic"))); - }); - insertMockFiles(157784760000L, 157784760000L); // Insert 2 mock files (0.9 and 0.13) with old timestamps so pruning should trigger on them. - - assertDoesNotThrow(() -> { - Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic"))); - Assertions - .assertEquals(2, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); - Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); - Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.13"))); - Assertions.assertTrue(config.getPruneOffset() >= 300000L); // Fails the test if the config is not correct. - Assertions.assertTrue((System.currentTimeMillis() - config.getPruneOffset()) > 157784760000L); - config.setMaximumFileSize(30000); - HdfsDataIngestion hdfsDataIngestion = new HdfsDataIngestion(config); - Thread.sleep(10000); - hdfsDataIngestion.run(); - - // hdfsDataIngestion.run(); has called fs.close() after finishing ingesting the records from kafka. Rebuild fs. - String hdfsURI = "hdfs://localhost:" + hdfsCluster.getNameNodePort() + "/"; - config.setHdfsuri(hdfsURI); - // ====== Init HDFS File System Object - Configuration fsConf = new Configuration(); - // Set FileSystem URI - fsConf.set("fs.defaultFS", hdfsURI); - // Because of Maven - fsConf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); - fsConf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); - // Set HADOOP user - System.setProperty("HADOOP_USER_NAME", "hdfs"); - System.setProperty("hadoop.home.dir", "/"); - fs = FileSystem.get(URI.create(hdfsURI), fsConf); - - // Assert that the kafka records were ingested and pruned correctly and the database holds only the expected 9 files. - Assertions - .assertEquals(9, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); - Assertions.assertFalse(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); - Assertions - .assertFalse(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.13"))); - Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "1.13"))); - Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "2.13"))); - Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "3.13"))); - Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "4.13"))); - Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "5.13"))); - Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "6.13"))); - Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "7.13"))); - Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "8.13"))); - Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "9.13"))); - }); - } - - @Test - public void ingestionTest1Old1NewFile() { - /* 14 records are inserted to HDFS database before starting ingestion, with 124/140 records in mock kafka consumer ready for ingestion. - Partitions through 1 to 9 will have only a single file, partition 0 will have 2 files (0.9 and 0.13). - partition 0 files are pre-made and inserted to the HDFS database with old timestamp for file 0.9 and new for 0.13.*/ - assertDoesNotThrow(() -> { - Assertions.assertFalse(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic"))); - }); - insertMockFiles(157784760000L, -1); // Insert 2 mock files (0.9 and 0.13) with old timestamp on 0.9 and new timestamp on 0.13. - - assertDoesNotThrow(() -> { - Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic"))); - Assertions - .assertEquals(2, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); - Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); - Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.13"))); - Assertions.assertTrue(config.getPruneOffset() >= 300000L); // Fails the test if the config is not correct. - Assertions.assertTrue((System.currentTimeMillis() - config.getPruneOffset()) > 157784760000L); - config.setMaximumFileSize(30000); - HdfsDataIngestion hdfsDataIngestion = new HdfsDataIngestion(config); - Thread.sleep(10000); - hdfsDataIngestion.run(); - - // hdfsDataIngestion.run(); has called fs.close() after finishing ingesting the records from kafka. Rebuild fs. - String hdfsURI = "hdfs://localhost:" + hdfsCluster.getNameNodePort() + "/"; - config.setHdfsuri(hdfsURI); - // ====== Init HDFS File System Object - Configuration fsConf = new Configuration(); - // Set FileSystem URI - fsConf.set("fs.defaultFS", hdfsURI); - // Because of Maven - fsConf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); - fsConf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); - // Set HADOOP user - System.setProperty("HADOOP_USER_NAME", "hdfs"); - System.setProperty("hadoop.home.dir", "/"); - fs = FileSystem.get(URI.create(hdfsURI), fsConf); - - // Assert that the kafka records were ingested and pruned correctly and the database holds only the expected 10 files. - Assertions - .assertEquals(10, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); - Assertions.assertFalse(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); - Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.13"))); - Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "1.13"))); - Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "2.13"))); - Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "3.13"))); - Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "4.13"))); - Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "5.13"))); - Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "6.13"))); - Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "7.13"))); - Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "8.13"))); - Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "9.13"))); - }); - } - - @Test - public void ingestionTest2NewFiles() { + public void ingestion2NewFilesTest() { // 14 records are inserted to HDFS database before starting ingestion, with 124/140 records in mock kafka consumer ready for ingestion. Partitions through 1 to 9 will have only a single file, partition 0 will have 2 files (0.9 and 0.13). - assertDoesNotThrow(() -> { - Assertions.assertFalse(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic"))); - }); - insertMockFiles(-1, -1); // Insert 2 mock files (0.9 and 0.13) with new timestamps so pruning should not trigger on them. assertDoesNotThrow(() -> { + // Assert the known starting state. Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic"))); Assertions .assertEquals(2, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); diff --git a/src/test/java/com/teragrep/cfe_39/Ingestion2OldFilesTest.java b/src/test/java/com/teragrep/cfe_39/Ingestion2OldFilesTest.java new file mode 100644 index 00000000..3f55c2fa --- /dev/null +++ b/src/test/java/com/teragrep/cfe_39/Ingestion2OldFilesTest.java @@ -0,0 +1,218 @@ +/* + * HDFS Data Ingestion for PTH_06 use CFE-39 + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. + */ +package com.teragrep.cfe_39; + +import com.teragrep.cfe_39.consumers.kafka.HdfsDataIngestion; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FileUtil; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.net.URI; +import java.nio.file.Files; +import java.util.Objects; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; + +public class Ingestion2OldFilesTest { + + private static final Logger LOGGER = LoggerFactory.getLogger(Ingestion2OldFilesTest.class); + private static MiniDFSCluster hdfsCluster; + private static File baseDir; + private static Config config; + private FileSystem fs; + + // Prepares known state for testing. + @BeforeEach + public void startMiniCluster() { + assertDoesNotThrow(() -> { + config = new Config(); + // Create a HDFS miniCluster + baseDir = Files.createTempDirectory("test_hdfs").toFile().getAbsoluteFile(); + Configuration conf = new Configuration(); + conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, baseDir.getAbsolutePath()); + MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf); + hdfsCluster = builder.build(); + String hdfsURI = "hdfs://localhost:" + hdfsCluster.getNameNodePort() + "/"; + config.setHdfsuri(hdfsURI); + DistributedFileSystem fileSystem = hdfsCluster.getFileSystem(); + + // ====== Init HDFS File System Object + Configuration fsConf = new Configuration(); + // Set FileSystem URI + fsConf.set("fs.defaultFS", hdfsURI); + // Because of Maven + fsConf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); + fsConf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); + // Set HADOOP user + System.setProperty("HADOOP_USER_NAME", "hdfs"); + System.setProperty("hadoop.home.dir", "/"); + fs = FileSystem.get(URI.create(hdfsURI), fsConf); + insertMockFiles(157784760000L, 157784760000L); // Insert 2 mock files (0.9 and 0.13) with old timestamps so pruning should trigger on them. + }); + } + + // Teardown the minicluster + @AfterEach + public void teardownMiniCluster() { + assertDoesNotThrow(() -> { + fs.close(); + }); + hdfsCluster.shutdown(); + FileUtil.fullyDelete(baseDir); + } + + @Test + public void ingestion2OldFilesTest() { + /* 14 records are inserted to HDFS database before starting ingestion, with 126/140 records in mock kafka consumer ready for ingestion. + Partitions through 1 to 9 will have only a single file, partition 0 will have 2 files (0.9 and 0.13). + partition 0 files are pre-made and inserted to the HDFS database with old timestamps that will mark them for pruning when ingestion is started.*/ + + assertDoesNotThrow(() -> { + // Assert the known starting state. + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic"))); + Assertions + .assertEquals(2, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.13"))); + Assertions.assertTrue(config.getPruneOffset() >= 300000L); // Fails the test if the config is not correct. + Assertions.assertTrue((System.currentTimeMillis() - config.getPruneOffset()) > 157784760000L); + config.setMaximumFileSize(30000); + HdfsDataIngestion hdfsDataIngestion = new HdfsDataIngestion(config); + Thread.sleep(10000); + hdfsDataIngestion.run(); + + // hdfsDataIngestion.run(); has called fs.close() after finishing ingesting the records from kafka. Rebuild fs. + String hdfsURI = "hdfs://localhost:" + hdfsCluster.getNameNodePort() + "/"; + config.setHdfsuri(hdfsURI); + // ====== Init HDFS File System Object + Configuration fsConf = new Configuration(); + // Set FileSystem URI + fsConf.set("fs.defaultFS", hdfsURI); + // Because of Maven + fsConf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); + fsConf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); + // Set HADOOP user + System.setProperty("HADOOP_USER_NAME", "hdfs"); + System.setProperty("hadoop.home.dir", "/"); + fs = FileSystem.get(URI.create(hdfsURI), fsConf); + + // Assert that the kafka records were ingested and pruned correctly and the database holds only the expected 9 files. + Assertions + .assertEquals(9, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); + Assertions.assertFalse(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); + Assertions + .assertFalse(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.13"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "1.13"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "2.13"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "3.13"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "4.13"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "5.13"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "6.13"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "7.13"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "8.13"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "9.13"))); + }); + } + + // Inserts pre-made avro-files to HDFS, which are normally generated during data ingestion from mock kafka consumer. + private void insertMockFiles(long fileTimestampA, long fileTimestampB) { + String path = config.getHdfsPath() + "/" + "testConsumerTopic"; // "hdfs:///opt/teragrep/cfe_39/srv/testConsumerTopic" + //Get the filesystem - HDFS + assertDoesNotThrow(() -> { + + // Sets the directory where the data should be stored, if the directory doesn't exist then it's created. + Path newDirectoryPath = new Path(path); + if (!fs.exists(newDirectoryPath)) { + // Create new Directory + fs.mkdirs(newDirectoryPath); + LOGGER.debug("Path {} created.", path); + } + + String dir = System.getProperty("user.dir") + "/src/test/java/com/teragrep/cfe_39/mockHdfsFiles"; + Set listOfFiles = Stream + .of(Objects.requireNonNull(new File(dir).listFiles())) + .filter(file -> !file.isDirectory()) + .map(File::getName) + .collect(Collectors.toSet()); + // Loop through all the avro files + for (String fileName : listOfFiles) { + String pathname = dir + "/" + fileName; + File avroFile = new File(pathname); + //==== Write file + LOGGER.debug("Begin Write file into hdfs"); + //Create a path + Path hdfswritepath = new Path(newDirectoryPath + "/" + avroFile.getName()); // filename should be set according to the requirements: 0.12345 where 0 is Kafka partition and 12345 is Kafka offset. + Assertions.assertFalse(fs.exists(hdfswritepath)); + Path readPath = new Path(avroFile.getPath()); + fs.copyFromLocalFile(readPath, hdfswritepath); + // Set fileTimestampA/fileTimestampB to something like 157784760000 to trigger pruning, -1 to not alter the timestamp. + if (Objects.equals(hdfswritepath.toString(), "hdfs:/opt/teragrep/cfe_39/srv/testConsumerTopic/0.9")) { + fs.setTimes(hdfswritepath, fileTimestampA, -1); + } + else if ( + Objects.equals(hdfswritepath.toString(), "hdfs:/opt/teragrep/cfe_39/srv/testConsumerTopic/0.13") + ) { + fs.setTimes(hdfswritepath, fileTimestampB, -1); + } + LOGGER.debug("End Write file into hdfs"); + LOGGER.debug("\nFile committed to HDFS, file writepath should be: {}\n", hdfswritepath); + } + }); + } +} From 95d01d59f793672aa22c8973d2328f249c240762 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Tue, 18 Jun 2024 11:00:23 +0300 Subject: [PATCH 097/146] Separated tests in PruningTest.java to PruningNoFilesTest.java, PruningOneNewFileTest.java, PruningOneOldFileTest.java, PruningOneOldOneNewFileTest.java, PruningTwoNewFilesTest.java and PruningTwoOldFilesTest.java --- .../teragrep/cfe_39/PruningNoFilesTest.java | 128 +++++++ .../cfe_39/PruningOneNewFileTest.java | 176 ++++++++++ .../cfe_39/PruningOneOldFileTest.java | 176 ++++++++++ .../cfe_39/PruningOneOldOneNewFileTest.java | 187 ++++++++++ .../java/com/teragrep/cfe_39/PruningTest.java | 331 ------------------ .../cfe_39/PruningTwoNewFilesTest.java | 185 ++++++++++ .../cfe_39/PruningTwoOldFilesTest.java | 188 ++++++++++ 7 files changed, 1040 insertions(+), 331 deletions(-) create mode 100644 src/test/java/com/teragrep/cfe_39/PruningNoFilesTest.java create mode 100644 src/test/java/com/teragrep/cfe_39/PruningOneNewFileTest.java create mode 100644 src/test/java/com/teragrep/cfe_39/PruningOneOldFileTest.java create mode 100644 src/test/java/com/teragrep/cfe_39/PruningOneOldOneNewFileTest.java delete mode 100644 src/test/java/com/teragrep/cfe_39/PruningTest.java create mode 100644 src/test/java/com/teragrep/cfe_39/PruningTwoNewFilesTest.java create mode 100644 src/test/java/com/teragrep/cfe_39/PruningTwoOldFilesTest.java diff --git a/src/test/java/com/teragrep/cfe_39/PruningNoFilesTest.java b/src/test/java/com/teragrep/cfe_39/PruningNoFilesTest.java new file mode 100644 index 00000000..671e84ea --- /dev/null +++ b/src/test/java/com/teragrep/cfe_39/PruningNoFilesTest.java @@ -0,0 +1,128 @@ +/* + * HDFS Data Ingestion for PTH_06 use CFE-39 + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. + */ +package com.teragrep.cfe_39; + +import com.teragrep.cfe_39.consumers.kafka.HDFSPrune; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FileUtil; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.net.URI; +import java.nio.file.Files; + +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; + +public class PruningNoFilesTest { + + private static final Logger LOGGER = LoggerFactory.getLogger(PruningNoFilesTest.class); + private static MiniDFSCluster hdfsCluster; + private static File baseDir; + private static Config config; + private FileSystem fs; + + // Start minicluster and initialize config. + @BeforeEach + public void startMiniCluster() { + assertDoesNotThrow(() -> { + config = new Config(); + // Create a HDFS miniCluster + baseDir = Files.createTempDirectory("test_hdfs").toFile().getAbsoluteFile(); + Configuration conf = new Configuration(); + conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, baseDir.getAbsolutePath()); + MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf); + hdfsCluster = builder.build(); + String hdfsURI = "hdfs://localhost:" + hdfsCluster.getNameNodePort() + "/"; + config.setHdfsuri(hdfsURI); + DistributedFileSystem fileSystem = hdfsCluster.getFileSystem(); + + // ====== Init HDFS File System Object + Configuration fsConf = new Configuration(); + // Set FileSystem URI + fsConf.set("fs.defaultFS", hdfsURI); + // Because of Maven + fsConf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); + fsConf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); + // Set HADOOP user + System.setProperty("HADOOP_USER_NAME", "hdfs"); + System.setProperty("hadoop.home.dir", "/"); + fs = FileSystem.get(URI.create(hdfsURI), fsConf); + }); + + } + + // Teardown the minicluster + @AfterEach + public void teardownMiniCluster() { + assertDoesNotThrow(fs::close); + hdfsCluster.shutdown(); + FileUtil.fullyDelete(baseDir); + } + + @Test + public void noFiles() { + assertDoesNotThrow(() -> { + Assertions.assertFalse(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic"))); + HDFSPrune hdfsPrune = new HDFSPrune(config, "testConsumerTopic"); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic"))); + Assertions + .assertEquals(0, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); + int deleted = hdfsPrune.prune(); + Assertions.assertEquals(0, deleted); + Assertions + .assertEquals(0, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); + }); + } +} diff --git a/src/test/java/com/teragrep/cfe_39/PruningOneNewFileTest.java b/src/test/java/com/teragrep/cfe_39/PruningOneNewFileTest.java new file mode 100644 index 00000000..0dcb0b25 --- /dev/null +++ b/src/test/java/com/teragrep/cfe_39/PruningOneNewFileTest.java @@ -0,0 +1,176 @@ +/* + * HDFS Data Ingestion for PTH_06 use CFE-39 + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. + */ +package com.teragrep.cfe_39; + +import com.teragrep.cfe_39.consumers.kafka.HDFSPrune; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FileUtil; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.net.URI; +import java.nio.file.Files; +import java.util.Objects; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; + +public class PruningOneNewFileTest { + + private static final Logger LOGGER = LoggerFactory.getLogger(PruningOneNewFileTest.class); + private static MiniDFSCluster hdfsCluster; + private static File baseDir; + private static Config config; + private FileSystem fs; + + // Prepares known state for testing. + @BeforeEach + public void startMiniCluster() { + assertDoesNotThrow(() -> { + config = new Config(); + // Create a HDFS miniCluster + baseDir = Files.createTempDirectory("test_hdfs").toFile().getAbsoluteFile(); + Configuration conf = new Configuration(); + conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, baseDir.getAbsolutePath()); + MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf); + hdfsCluster = builder.build(); + String hdfsURI = "hdfs://localhost:" + hdfsCluster.getNameNodePort() + "/"; + config.setHdfsuri(hdfsURI); + DistributedFileSystem fileSystem = hdfsCluster.getFileSystem(); + + // ====== Init HDFS File System Object + Configuration fsConf = new Configuration(); + // Set FileSystem URI + fsConf.set("fs.defaultFS", hdfsURI); + // Because of Maven + fsConf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); + fsConf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); + // Set HADOOP user + System.setProperty("HADOOP_USER_NAME", "hdfs"); + System.setProperty("hadoop.home.dir", "/"); + fs = FileSystem.get(URI.create(hdfsURI), fsConf); + insertMockFile(-1); // Insert 1 new mock file. + }); + + } + + // Teardown the minicluster + @AfterEach + public void teardownMiniCluster() { + assertDoesNotThrow(fs::close); + hdfsCluster.shutdown(); + FileUtil.fullyDelete(baseDir); + } + + @Test + public void oneNewFileTest() { + // Test for not triggering pruning for a single file. + Assertions.assertTrue(config.getPruneOffset() >= 300000L); // Fails the test if the config is not correct, too low pruning offset can prune the files if the test is lagging. + Assertions.assertTrue(System.currentTimeMillis() - config.getPruneOffset() > 157784760000L); + + assertDoesNotThrow(() -> { + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic"))); + Assertions + .assertEquals(1, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); + HDFSPrune hdfsPrune = new HDFSPrune(config, "testConsumerTopic"); + int deleted = hdfsPrune.prune(); + Assertions.assertEquals(0, deleted); + // Also check with HDFS access if expected files still exist. + Assertions + .assertEquals(1, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); + }); + } + + // Inserts a single pre-made avro-file to HDFS, which is normally generated during data ingestion from mock kafka consumer. + private void insertMockFile(long fileTimestamp) { + String path = config.getHdfsPath() + "/" + "testConsumerTopic"; // "hdfs:///opt/teragrep/cfe_39/srv/testConsumerTopic" + assertDoesNotThrow(() -> { + + // Sets the directory where the data should be stored, if the directory doesn't exist then it's created. + Path newDirectoryPath = new Path(path); + if (!fs.exists(newDirectoryPath)) { + // Create new Directory + fs.mkdirs(newDirectoryPath); + LOGGER.debug("Path {} created.", path); + } + + String dir = System.getProperty("user.dir") + "/src/test/java/com/teragrep/cfe_39/mockHdfsFiles"; + Set listOfFiles = Stream + .of(Objects.requireNonNull(new File(dir).listFiles())) + .filter(file -> !file.isDirectory()) + .map(File::getName) + .collect(Collectors.toSet()); + String fileName = "0.9"; + Assertions.assertTrue(listOfFiles.contains(fileName)); + String pathname = dir + "/" + fileName; + File avroFile = new File(pathname); + //==== Write file + LOGGER.debug("Begin Write file into hdfs"); + //Create a path + Path hdfswritepath = new Path(newDirectoryPath + "/" + avroFile.getName()); // filename should be set according to the requirements: 0.12345 where 0 is Kafka partition and 12345 is Kafka offset. + Assertions.assertFalse(fs.exists(hdfswritepath)); + Path readPath = new Path(avroFile.getPath()); + fs.copyFromLocalFile(readPath, hdfswritepath); + // Set fileTimestamp to something like 157784760000 to trigger pruning, -1 to not alter the timestamp. + fs.setTimes(hdfswritepath, fileTimestamp, -1); + LOGGER.debug("End Write file into hdfs"); + LOGGER.debug("\nFile committed to HDFS, file writepath should be: {}\n", hdfswritepath); + }); + } +} diff --git a/src/test/java/com/teragrep/cfe_39/PruningOneOldFileTest.java b/src/test/java/com/teragrep/cfe_39/PruningOneOldFileTest.java new file mode 100644 index 00000000..a62140cf --- /dev/null +++ b/src/test/java/com/teragrep/cfe_39/PruningOneOldFileTest.java @@ -0,0 +1,176 @@ +/* + * HDFS Data Ingestion for PTH_06 use CFE-39 + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. + */ +package com.teragrep.cfe_39; + +import com.teragrep.cfe_39.consumers.kafka.HDFSPrune; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FileUtil; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.net.URI; +import java.nio.file.Files; +import java.util.Objects; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; + +public class PruningOneOldFileTest { + + private static final Logger LOGGER = LoggerFactory.getLogger(PruningOneOldFileTest.class); + private static MiniDFSCluster hdfsCluster; + private static File baseDir; + private static Config config; + private FileSystem fs; + + // Prepares known state for testing. + @BeforeEach + public void startMiniCluster() { + assertDoesNotThrow(() -> { + config = new Config(); + // Create a HDFS miniCluster + baseDir = Files.createTempDirectory("test_hdfs").toFile().getAbsoluteFile(); + Configuration conf = new Configuration(); + conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, baseDir.getAbsolutePath()); + MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf); + hdfsCluster = builder.build(); + String hdfsURI = "hdfs://localhost:" + hdfsCluster.getNameNodePort() + "/"; + config.setHdfsuri(hdfsURI); + DistributedFileSystem fileSystem = hdfsCluster.getFileSystem(); + + // ====== Init HDFS File System Object + Configuration fsConf = new Configuration(); + // Set FileSystem URI + fsConf.set("fs.defaultFS", hdfsURI); + // Because of Maven + fsConf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); + fsConf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); + // Set HADOOP user + System.setProperty("HADOOP_USER_NAME", "hdfs"); + System.setProperty("hadoop.home.dir", "/"); + fs = FileSystem.get(URI.create(hdfsURI), fsConf); + insertMockFile(157784760000L); // Insert 1 old mock file. + }); + + } + + // Teardown the minicluster + @AfterEach + public void teardownMiniCluster() { + assertDoesNotThrow(fs::close); + hdfsCluster.shutdown(); + FileUtil.fullyDelete(baseDir); + } + + @Test + public void oneOldFileTest() { + // Test for triggering pruning for a single file. + Assertions.assertTrue(config.getPruneOffset() >= 300000L); // Fails the test if the config is not correct, too low pruning offset can prune the files if the test is lagging. + Assertions.assertTrue(System.currentTimeMillis() - config.getPruneOffset() > 157784760000L); + + assertDoesNotThrow(() -> { + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic"))); + Assertions + .assertEquals(1, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); + HDFSPrune hdfsPrune = new HDFSPrune(config, "testConsumerTopic"); + int deleted = hdfsPrune.prune(); + Assertions.assertEquals(1, deleted); + // Also check with HDFS access if expected files still exist. + Assertions + .assertEquals(0, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); + Assertions.assertFalse(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); + }); + } + + // Inserts a single pre-made avro-file to HDFS, which is normally generated during data ingestion from mock kafka consumer. + private void insertMockFile(long fileTimestamp) { + String path = config.getHdfsPath() + "/" + "testConsumerTopic"; // "hdfs:///opt/teragrep/cfe_39/srv/testConsumerTopic" + assertDoesNotThrow(() -> { + + // Sets the directory where the data should be stored, if the directory doesn't exist then it's created. + Path newDirectoryPath = new Path(path); + if (!fs.exists(newDirectoryPath)) { + // Create new Directory + fs.mkdirs(newDirectoryPath); + LOGGER.debug("Path {} created.", path); + } + + String dir = System.getProperty("user.dir") + "/src/test/java/com/teragrep/cfe_39/mockHdfsFiles"; + Set listOfFiles = Stream + .of(Objects.requireNonNull(new File(dir).listFiles())) + .filter(file -> !file.isDirectory()) + .map(File::getName) + .collect(Collectors.toSet()); + String fileName = "0.9"; + Assertions.assertTrue(listOfFiles.contains(fileName)); + String pathname = dir + "/" + fileName; + File avroFile = new File(pathname); + //==== Write file + LOGGER.debug("Begin Write file into hdfs"); + //Create a path + Path hdfswritepath = new Path(newDirectoryPath + "/" + avroFile.getName()); // filename should be set according to the requirements: 0.12345 where 0 is Kafka partition and 12345 is Kafka offset. + Assertions.assertFalse(fs.exists(hdfswritepath)); + Path readPath = new Path(avroFile.getPath()); + fs.copyFromLocalFile(readPath, hdfswritepath); + // Set fileTimestamp to something like 157784760000 to trigger pruning, -1 to not alter the timestamp. + fs.setTimes(hdfswritepath, fileTimestamp, -1); + LOGGER.debug("End Write file into hdfs"); + LOGGER.debug("\nFile committed to HDFS, file writepath should be: {}\n", hdfswritepath); + }); + } +} diff --git a/src/test/java/com/teragrep/cfe_39/PruningOneOldOneNewFileTest.java b/src/test/java/com/teragrep/cfe_39/PruningOneOldOneNewFileTest.java new file mode 100644 index 00000000..655e0852 --- /dev/null +++ b/src/test/java/com/teragrep/cfe_39/PruningOneOldOneNewFileTest.java @@ -0,0 +1,187 @@ +/* + * HDFS Data Ingestion for PTH_06 use CFE-39 + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. + */ +package com.teragrep.cfe_39; + +import com.teragrep.cfe_39.consumers.kafka.HDFSPrune; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FileUtil; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.net.URI; +import java.nio.file.Files; +import java.util.Objects; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; + +public class PruningOneOldOneNewFileTest { + + private static final Logger LOGGER = LoggerFactory.getLogger(PruningOneOldOneNewFileTest.class); + private static MiniDFSCluster hdfsCluster; + private static File baseDir; + private static Config config; + private FileSystem fs; + + // Prepares known state for testing. + @BeforeEach + public void startMiniCluster() { + assertDoesNotThrow(() -> { + config = new Config(); + // Create a HDFS miniCluster + baseDir = Files.createTempDirectory("test_hdfs").toFile().getAbsoluteFile(); + Configuration conf = new Configuration(); + conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, baseDir.getAbsolutePath()); + MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf); + hdfsCluster = builder.build(); + String hdfsURI = "hdfs://localhost:" + hdfsCluster.getNameNodePort() + "/"; + config.setHdfsuri(hdfsURI); + DistributedFileSystem fileSystem = hdfsCluster.getFileSystem(); + + // ====== Init HDFS File System Object + Configuration fsConf = new Configuration(); + // Set FileSystem URI + fsConf.set("fs.defaultFS", hdfsURI); + // Because of Maven + fsConf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); + fsConf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); + // Set HADOOP user + System.setProperty("HADOOP_USER_NAME", "hdfs"); + System.setProperty("hadoop.home.dir", "/"); + fs = FileSystem.get(URI.create(hdfsURI), fsConf); + insertMockFiles(157784760000L, -1); // Insert 2 mock files, one with old timestamp and one with new timestamp. + }); + + } + + // Teardown the minicluster + @AfterEach + public void teardownMiniCluster() { + assertDoesNotThrow(fs::close); + hdfsCluster.shutdown(); + FileUtil.fullyDelete(baseDir); + } + + @Test + public void oneOldOneNewFileTest() { + // Test for triggering pruning for 1 out of 2 files in the topic. + Assertions.assertTrue(config.getPruneOffset() >= 300000L); // Fails the test if the config is not correct, too low pruning offset can prune the files if the test is lagging. + Assertions.assertTrue(System.currentTimeMillis() - config.getPruneOffset() > 157784760000L); + + assertDoesNotThrow(() -> { + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic"))); + Assertions + .assertEquals(2, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.13"))); + HDFSPrune hdfsPrune = new HDFSPrune(config, "testConsumerTopic"); + int deleted = hdfsPrune.prune(); + Assertions.assertEquals(1, deleted); + // Also check with HDFS access if expected files still exist. + Assertions + .assertEquals(1, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); + Assertions.assertFalse(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.13"))); + }); + } + + // Inserts pre-made avro-files to HDFS, which are normally generated during data ingestion from mock kafka consumer. + private void insertMockFiles(long fileTimestampA, long fileTimestampB) { + String path = config.getHdfsPath() + "/" + "testConsumerTopic"; // "hdfs:///opt/teragrep/cfe_39/srv/testConsumerTopic" + //Get the filesystem - HDFS + assertDoesNotThrow(() -> { + + // Sets the directory where the data should be stored, if the directory doesn't exist then it's created. + Path newDirectoryPath = new Path(path); + if (!fs.exists(newDirectoryPath)) { + // Create new Directory + fs.mkdirs(newDirectoryPath); + LOGGER.debug("Path {} created.", path); + } + + String dir = System.getProperty("user.dir") + "/src/test/java/com/teragrep/cfe_39/mockHdfsFiles"; + Set listOfFiles = Stream + .of(Objects.requireNonNull(new File(dir).listFiles())) + .filter(file -> !file.isDirectory()) + .map(File::getName) + .collect(Collectors.toSet()); + // Loop through all the avro files + for (String fileName : listOfFiles) { + String pathname = dir + "/" + fileName; + File avroFile = new File(pathname); + //==== Write file + LOGGER.debug("Begin Write file into hdfs"); + //Create a path + Path hdfswritepath = new Path(newDirectoryPath + "/" + avroFile.getName()); // filename should be set according to the requirements: 0.12345 where 0 is Kafka partition and 12345 is Kafka offset. + Assertions.assertFalse(fs.exists(hdfswritepath)); + Path readPath = new Path(avroFile.getPath()); + fs.copyFromLocalFile(readPath, hdfswritepath); + // Set fileTimestampA/fileTimestampB to something like 157784760000 to trigger pruning, -1 to not alter the timestamp. + if (Objects.equals(hdfswritepath.toString(), "hdfs:/opt/teragrep/cfe_39/srv/testConsumerTopic/0.9")) { + fs.setTimes(hdfswritepath, fileTimestampA, -1); + } + else if ( + Objects.equals(hdfswritepath.toString(), "hdfs:/opt/teragrep/cfe_39/srv/testConsumerTopic/0.13") + ) { + fs.setTimes(hdfswritepath, fileTimestampB, -1); + } + LOGGER.debug("End Write file into hdfs"); + LOGGER.debug("\nFile committed to HDFS, file writepath should be: {}\n", hdfswritepath); + } + }); + } +} diff --git a/src/test/java/com/teragrep/cfe_39/PruningTest.java b/src/test/java/com/teragrep/cfe_39/PruningTest.java deleted file mode 100644 index 37405276..00000000 --- a/src/test/java/com/teragrep/cfe_39/PruningTest.java +++ /dev/null @@ -1,331 +0,0 @@ -/* - * HDFS Data Ingestion for PTH_06 use CFE-39 - * Copyright (C) 2021-2024 Suomen Kanuuna Oy - * - * This program is free software: you can redistribute it and/or modify - * it under the terms of the GNU Affero General Public License as published by - * the Free Software Foundation, either version 3 of the License, or - * (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU Affero General Public License for more details. - * - * You should have received a copy of the GNU Affero General Public License - * along with this program. If not, see . - * - * - * Additional permission under GNU Affero General Public License version 3 - * section 7 - * - * If you modify this Program, or any covered work, by linking or combining it - * with other code, such other code is not for that reason alone subject to any - * of the requirements of the GNU Affero GPL version 3 as long as this Program - * is the same Program as licensed from Suomen Kanuuna Oy without any additional - * modifications. - * - * Supplemented terms under GNU Affero General Public License version 3 - * section 7 - * - * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified - * versions must be marked as "Modified version of" The Program. - * - * Names of the licensors and authors may not be used for publicity purposes. - * - * No rights are granted for use of trade names, trademarks, or service marks - * which are in The Program if any. - * - * Licensee must indemnify licensors and authors for any liability that these - * contractual assumptions impose on licensors and authors. - * - * To the extent this program is licensed as part of the Commercial versions of - * Teragrep, the applicable Commercial License may apply to this file if you as - * a licensee so wish it. - */ -package com.teragrep.cfe_39; - -import com.teragrep.cfe_39.consumers.kafka.HDFSPrune; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.FileUtil; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hdfs.DistributedFileSystem; -import org.apache.hadoop.hdfs.MiniDFSCluster; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.File; -import java.net.URI; -import java.nio.file.Files; -import java.util.Objects; -import java.util.Set; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; - -public class PruningTest { - - private static final Logger LOGGER = LoggerFactory.getLogger(PruningTest.class); - private static MiniDFSCluster hdfsCluster; - private static File baseDir; - private static Config config; - private FileSystem fs; - - // Start minicluster and initialize config. - @BeforeEach - public void startMiniCluster() { - assertDoesNotThrow(() -> { - config = new Config(); - // Create a HDFS miniCluster - baseDir = Files.createTempDirectory("test_hdfs").toFile().getAbsoluteFile(); - Configuration conf = new Configuration(); - conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, baseDir.getAbsolutePath()); - MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf); - hdfsCluster = builder.build(); - String hdfsURI = "hdfs://localhost:" + hdfsCluster.getNameNodePort() + "/"; - config.setHdfsuri(hdfsURI); - DistributedFileSystem fileSystem = hdfsCluster.getFileSystem(); - - // ====== Init HDFS File System Object - Configuration fsConf = new Configuration(); - // Set FileSystem URI - fsConf.set("fs.defaultFS", hdfsURI); - // Because of Maven - fsConf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); - fsConf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); - // Set HADOOP user - System.setProperty("HADOOP_USER_NAME", "hdfs"); - System.setProperty("hadoop.home.dir", "/"); - fs = FileSystem.get(URI.create(hdfsURI), fsConf); - }); - - } - - // Teardown the minicluster - @AfterEach - public void teardownMiniCluster() { - assertDoesNotThrow(fs::close); - hdfsCluster.shutdown(); - FileUtil.fullyDelete(baseDir); - } - - @Test - public void twoNewFiles() { - // Test for not triggering pruning for 2 files in the topic. - Assertions.assertTrue(config.getPruneOffset() >= 300000L); // Fails the test if the config is not correct, too low pruning offset can prune the files if the test is lagging. - insertMockFiles(-1, -1); // Insert 2 mock files with new timestamps so pruning should not trigger on them. - - assertDoesNotThrow(() -> { - Assertions - .assertEquals(2, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); - Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); - Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.13"))); - HDFSPrune hdfsPrune = new HDFSPrune(config, "testConsumerTopic"); - int deleted = hdfsPrune.prune(); - Assertions.assertEquals(0, deleted); - // Also check with HDFS access if expected files still exist. - Assertions - .assertEquals(2, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); - Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); - Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.13"))); - }); - } - - @Test - public void noFiles() { - assertDoesNotThrow(() -> { - Assertions.assertFalse(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic"))); - HDFSPrune hdfsPrune = new HDFSPrune(config, "testConsumerTopic"); - Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic"))); - Assertions - .assertEquals(0, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); - int deleted = hdfsPrune.prune(); - Assertions.assertEquals(0, deleted); - Assertions - .assertEquals(0, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); - }); - } - - @Test - public void twoOldFiles() { - // Test for triggering pruning for 2 files in the topic. - Assertions.assertTrue(config.getPruneOffset() >= 300000L); // Fails the test if the config is not correct, too low pruning offset can prune the files if the test is lagging. - Assertions.assertTrue(System.currentTimeMillis() - config.getPruneOffset() > 157784760000L); - insertMockFiles(157784760000L, 157784760000L); // Insert 2 mock files with old timestamps so pruning should trigger on them. - - assertDoesNotThrow(() -> { - Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic"))); - Assertions - .assertEquals(2, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); - Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); - Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.13"))); - HDFSPrune hdfsPrune = new HDFSPrune(config, "testConsumerTopic"); - int deleted = hdfsPrune.prune(); - Assertions.assertEquals(2, deleted); - // Also check with HDFS access if expected files still exist. - Assertions - .assertEquals(0, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); - Assertions.assertFalse(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); - Assertions - .assertFalse(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.13"))); - }); - } - - @Test - public void oneOldOneNewFile() { - // Test for triggering pruning for 1 out of 2 files in the topic. - Assertions.assertTrue(config.getPruneOffset() >= 300000L); // Fails the test if the config is not correct, too low pruning offset can prune the files if the test is lagging. - Assertions.assertTrue(System.currentTimeMillis() - config.getPruneOffset() > 157784760000L); - insertMockFiles(157784760000L, -1); // Insert 2 mock files, one with old timestamp and one with new timestamp. - - assertDoesNotThrow(() -> { - Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic"))); - Assertions - .assertEquals(2, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); - Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); - Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.13"))); - HDFSPrune hdfsPrune = new HDFSPrune(config, "testConsumerTopic"); - int deleted = hdfsPrune.prune(); - Assertions.assertEquals(1, deleted); - // Also check with HDFS access if expected files still exist. - Assertions - .assertEquals(1, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); - Assertions.assertFalse(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); - Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.13"))); - }); - } - - @Test - public void oneOldFile() { - // Test for triggering pruning for a single file. - Assertions.assertTrue(config.getPruneOffset() >= 300000L); // Fails the test if the config is not correct, too low pruning offset can prune the files if the test is lagging. - Assertions.assertTrue(System.currentTimeMillis() - config.getPruneOffset() > 157784760000L); - insertMockFile(157784760000L); // Insert 1 old mock file. - - assertDoesNotThrow(() -> { - Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic"))); - Assertions - .assertEquals(1, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); - Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); - HDFSPrune hdfsPrune = new HDFSPrune(config, "testConsumerTopic"); - int deleted = hdfsPrune.prune(); - Assertions.assertEquals(1, deleted); - // Also check with HDFS access if expected files still exist. - Assertions - .assertEquals(0, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); - Assertions.assertFalse(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); - }); - } - - @Test - public void oneNewFile() { - // Test for not triggering pruning for a single file. - Assertions.assertTrue(config.getPruneOffset() >= 300000L); // Fails the test if the config is not correct, too low pruning offset can prune the files if the test is lagging. - Assertions.assertTrue(System.currentTimeMillis() - config.getPruneOffset() > 157784760000L); - insertMockFile(-1); // Insert 1 new mock file. - - assertDoesNotThrow(() -> { - Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic"))); - Assertions - .assertEquals(1, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); - Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); - HDFSPrune hdfsPrune = new HDFSPrune(config, "testConsumerTopic"); - int deleted = hdfsPrune.prune(); - Assertions.assertEquals(0, deleted); - // Also check with HDFS access if expected files still exist. - Assertions - .assertEquals(1, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); - Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); - }); - } - - // Inserts pre-made avro-files to HDFS, which are normally generated during data ingestion from mock kafka consumer. - private void insertMockFiles(long fileTimestampA, long fileTimestampB) { - String path = config.getHdfsPath() + "/" + "testConsumerTopic"; // "hdfs:///opt/teragrep/cfe_39/srv/testConsumerTopic" - //Get the filesystem - HDFS - assertDoesNotThrow(() -> { - - // Sets the directory where the data should be stored, if the directory doesn't exist then it's created. - Path newDirectoryPath = new Path(path); - if (!fs.exists(newDirectoryPath)) { - // Create new Directory - fs.mkdirs(newDirectoryPath); - LOGGER.debug("Path {} created.", path); - } - - String dir = System.getProperty("user.dir") + "/src/test/java/com/teragrep/cfe_39/mockHdfsFiles"; - Set listOfFiles = Stream - .of(Objects.requireNonNull(new File(dir).listFiles())) - .filter(file -> !file.isDirectory()) - .map(File::getName) - .collect(Collectors.toSet()); - // Loop through all the avro files - for (String fileName : listOfFiles) { - String pathname = dir + "/" + fileName; - File avroFile = new File(pathname); - //==== Write file - LOGGER.debug("Begin Write file into hdfs"); - //Create a path - Path hdfswritepath = new Path(newDirectoryPath + "/" + avroFile.getName()); // filename should be set according to the requirements: 0.12345 where 0 is Kafka partition and 12345 is Kafka offset. - Assertions.assertFalse(fs.exists(hdfswritepath)); - Path readPath = new Path(avroFile.getPath()); - fs.copyFromLocalFile(readPath, hdfswritepath); - // Set fileTimestampA/fileTimestampB to something like 157784760000 to trigger pruning, -1 to not alter the timestamp. - if (Objects.equals(hdfswritepath.toString(), "hdfs:/opt/teragrep/cfe_39/srv/testConsumerTopic/0.9")) { - fs.setTimes(hdfswritepath, fileTimestampA, -1); - } - else if ( - Objects.equals(hdfswritepath.toString(), "hdfs:/opt/teragrep/cfe_39/srv/testConsumerTopic/0.13") - ) { - fs.setTimes(hdfswritepath, fileTimestampB, -1); - } - LOGGER.debug("End Write file into hdfs"); - LOGGER.debug("\nFile committed to HDFS, file writepath should be: {}\n", hdfswritepath); - } - }); - } - - // Inserts a single pre-made avro-file to HDFS, which is normally generated during data ingestion from mock kafka consumer. - private void insertMockFile(long fileTimestamp) { - String path = config.getHdfsPath() + "/" + "testConsumerTopic"; // "hdfs:///opt/teragrep/cfe_39/srv/testConsumerTopic" - assertDoesNotThrow(() -> { - - // Sets the directory where the data should be stored, if the directory doesn't exist then it's created. - Path newDirectoryPath = new Path(path); - if (!fs.exists(newDirectoryPath)) { - // Create new Directory - fs.mkdirs(newDirectoryPath); - LOGGER.debug("Path {} created.", path); - } - - String dir = System.getProperty("user.dir") + "/src/test/java/com/teragrep/cfe_39/mockHdfsFiles"; - Set listOfFiles = Stream - .of(Objects.requireNonNull(new File(dir).listFiles())) - .filter(file -> !file.isDirectory()) - .map(File::getName) - .collect(Collectors.toSet()); - String fileName = "0.9"; - Assertions.assertTrue(listOfFiles.contains(fileName)); - String pathname = dir + "/" + fileName; - File avroFile = new File(pathname); - //==== Write file - LOGGER.debug("Begin Write file into hdfs"); - //Create a path - Path hdfswritepath = new Path(newDirectoryPath + "/" + avroFile.getName()); // filename should be set according to the requirements: 0.12345 where 0 is Kafka partition and 12345 is Kafka offset. - Assertions.assertFalse(fs.exists(hdfswritepath)); - Path readPath = new Path(avroFile.getPath()); - fs.copyFromLocalFile(readPath, hdfswritepath); - // Set fileTimestamp to something like 157784760000 to trigger pruning, -1 to not alter the timestamp. - fs.setTimes(hdfswritepath, fileTimestamp, -1); - LOGGER.debug("End Write file into hdfs"); - LOGGER.debug("\nFile committed to HDFS, file writepath should be: {}\n", hdfswritepath); - }); - } - -} diff --git a/src/test/java/com/teragrep/cfe_39/PruningTwoNewFilesTest.java b/src/test/java/com/teragrep/cfe_39/PruningTwoNewFilesTest.java new file mode 100644 index 00000000..1d66875f --- /dev/null +++ b/src/test/java/com/teragrep/cfe_39/PruningTwoNewFilesTest.java @@ -0,0 +1,185 @@ +/* + * HDFS Data Ingestion for PTH_06 use CFE-39 + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. + */ +package com.teragrep.cfe_39; + +import com.teragrep.cfe_39.consumers.kafka.HDFSPrune; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FileUtil; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.net.URI; +import java.nio.file.Files; +import java.util.Objects; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; + +public class PruningTwoNewFilesTest { + + private static final Logger LOGGER = LoggerFactory.getLogger(PruningTwoNewFilesTest.class); + private static MiniDFSCluster hdfsCluster; + private static File baseDir; + private static Config config; + private FileSystem fs; + + // Prepares known state for testing. + @BeforeEach + public void startMiniCluster() { + assertDoesNotThrow(() -> { + config = new Config(); + // Create a HDFS miniCluster + baseDir = Files.createTempDirectory("test_hdfs").toFile().getAbsoluteFile(); + Configuration conf = new Configuration(); + conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, baseDir.getAbsolutePath()); + MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf); + hdfsCluster = builder.build(); + String hdfsURI = "hdfs://localhost:" + hdfsCluster.getNameNodePort() + "/"; + config.setHdfsuri(hdfsURI); + DistributedFileSystem fileSystem = hdfsCluster.getFileSystem(); + + // ====== Init HDFS File System Object + Configuration fsConf = new Configuration(); + // Set FileSystem URI + fsConf.set("fs.defaultFS", hdfsURI); + // Because of Maven + fsConf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); + fsConf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); + // Set HADOOP user + System.setProperty("HADOOP_USER_NAME", "hdfs"); + System.setProperty("hadoop.home.dir", "/"); + fs = FileSystem.get(URI.create(hdfsURI), fsConf); + insertMockFiles(-1, -1); // Insert 2 mock files with new timestamps so pruning should not trigger on them. + }); + + } + + // Teardown the minicluster + @AfterEach + public void teardownMiniCluster() { + assertDoesNotThrow(fs::close); + hdfsCluster.shutdown(); + FileUtil.fullyDelete(baseDir); + } + + @Test + public void twoNewFilesTest() { + // Test for not triggering pruning for 2 files in the topic. + Assertions.assertTrue(config.getPruneOffset() >= 300000L); // Fails the test if the config is not correct, too low pruning offset can prune the files if the test is lagging. + + assertDoesNotThrow(() -> { + Assertions + .assertEquals(2, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.13"))); + HDFSPrune hdfsPrune = new HDFSPrune(config, "testConsumerTopic"); + int deleted = hdfsPrune.prune(); + Assertions.assertEquals(0, deleted); + // Also check with HDFS access if expected files still exist. + Assertions + .assertEquals(2, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.13"))); + }); + } + + // Inserts pre-made avro-files to HDFS, which are normally generated during data ingestion from mock kafka consumer. + private void insertMockFiles(long fileTimestampA, long fileTimestampB) { + String path = config.getHdfsPath() + "/" + "testConsumerTopic"; // "hdfs:///opt/teragrep/cfe_39/srv/testConsumerTopic" + //Get the filesystem - HDFS + assertDoesNotThrow(() -> { + + // Sets the directory where the data should be stored, if the directory doesn't exist then it's created. + Path newDirectoryPath = new Path(path); + if (!fs.exists(newDirectoryPath)) { + // Create new Directory + fs.mkdirs(newDirectoryPath); + LOGGER.debug("Path {} created.", path); + } + + String dir = System.getProperty("user.dir") + "/src/test/java/com/teragrep/cfe_39/mockHdfsFiles"; + Set listOfFiles = Stream + .of(Objects.requireNonNull(new File(dir).listFiles())) + .filter(file -> !file.isDirectory()) + .map(File::getName) + .collect(Collectors.toSet()); + // Loop through all the avro files + for (String fileName : listOfFiles) { + String pathname = dir + "/" + fileName; + File avroFile = new File(pathname); + //==== Write file + LOGGER.debug("Begin Write file into hdfs"); + //Create a path + Path hdfswritepath = new Path(newDirectoryPath + "/" + avroFile.getName()); // filename should be set according to the requirements: 0.12345 where 0 is Kafka partition and 12345 is Kafka offset. + Assertions.assertFalse(fs.exists(hdfswritepath)); + Path readPath = new Path(avroFile.getPath()); + fs.copyFromLocalFile(readPath, hdfswritepath); + // Set fileTimestampA/fileTimestampB to something like 157784760000 to trigger pruning, -1 to not alter the timestamp. + if (Objects.equals(hdfswritepath.toString(), "hdfs:/opt/teragrep/cfe_39/srv/testConsumerTopic/0.9")) { + fs.setTimes(hdfswritepath, fileTimestampA, -1); + } + else if ( + Objects.equals(hdfswritepath.toString(), "hdfs:/opt/teragrep/cfe_39/srv/testConsumerTopic/0.13") + ) { + fs.setTimes(hdfswritepath, fileTimestampB, -1); + } + LOGGER.debug("End Write file into hdfs"); + LOGGER.debug("\nFile committed to HDFS, file writepath should be: {}\n", hdfswritepath); + } + }); + } +} diff --git a/src/test/java/com/teragrep/cfe_39/PruningTwoOldFilesTest.java b/src/test/java/com/teragrep/cfe_39/PruningTwoOldFilesTest.java new file mode 100644 index 00000000..00dfb299 --- /dev/null +++ b/src/test/java/com/teragrep/cfe_39/PruningTwoOldFilesTest.java @@ -0,0 +1,188 @@ +/* + * HDFS Data Ingestion for PTH_06 use CFE-39 + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. + */ +package com.teragrep.cfe_39; + +import com.teragrep.cfe_39.consumers.kafka.HDFSPrune; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FileUtil; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.net.URI; +import java.nio.file.Files; +import java.util.Objects; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; + +public class PruningTwoOldFilesTest { + + private static final Logger LOGGER = LoggerFactory.getLogger(PruningTwoOldFilesTest.class); + private static MiniDFSCluster hdfsCluster; + private static File baseDir; + private static Config config; + private FileSystem fs; + + // Prepares known state for testing. + @BeforeEach + public void startMiniCluster() { + assertDoesNotThrow(() -> { + config = new Config(); + // Create a HDFS miniCluster + baseDir = Files.createTempDirectory("test_hdfs").toFile().getAbsoluteFile(); + Configuration conf = new Configuration(); + conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, baseDir.getAbsolutePath()); + MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf); + hdfsCluster = builder.build(); + String hdfsURI = "hdfs://localhost:" + hdfsCluster.getNameNodePort() + "/"; + config.setHdfsuri(hdfsURI); + DistributedFileSystem fileSystem = hdfsCluster.getFileSystem(); + + // ====== Init HDFS File System Object + Configuration fsConf = new Configuration(); + // Set FileSystem URI + fsConf.set("fs.defaultFS", hdfsURI); + // Because of Maven + fsConf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); + fsConf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); + // Set HADOOP user + System.setProperty("HADOOP_USER_NAME", "hdfs"); + System.setProperty("hadoop.home.dir", "/"); + fs = FileSystem.get(URI.create(hdfsURI), fsConf); + insertMockFiles(157784760000L, 157784760000L); // Insert 2 mock files with old timestamps so pruning should trigger on them. + }); + + } + + // Teardown the minicluster + @AfterEach + public void teardownMiniCluster() { + assertDoesNotThrow(fs::close); + hdfsCluster.shutdown(); + FileUtil.fullyDelete(baseDir); + } + + @Test + public void twoOldFilesTest() { + // Test for triggering pruning for 2 files in the topic. + Assertions.assertTrue(config.getPruneOffset() >= 300000L); // Fails the test if the config is not correct, too low pruning offset can prune the files if the test is lagging. + Assertions.assertTrue(System.currentTimeMillis() - config.getPruneOffset() > 157784760000L); + + assertDoesNotThrow(() -> { + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic"))); + Assertions + .assertEquals(2, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.13"))); + HDFSPrune hdfsPrune = new HDFSPrune(config, "testConsumerTopic"); + int deleted = hdfsPrune.prune(); + Assertions.assertEquals(2, deleted); + // Also check with HDFS access if expected files still exist. + Assertions + .assertEquals(0, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); + Assertions.assertFalse(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); + Assertions + .assertFalse(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.13"))); + }); + } + + // Inserts pre-made avro-files to HDFS, which are normally generated during data ingestion from mock kafka consumer. + private void insertMockFiles(long fileTimestampA, long fileTimestampB) { + String path = config.getHdfsPath() + "/" + "testConsumerTopic"; // "hdfs:///opt/teragrep/cfe_39/srv/testConsumerTopic" + //Get the filesystem - HDFS + assertDoesNotThrow(() -> { + + // Sets the directory where the data should be stored, if the directory doesn't exist then it's created. + Path newDirectoryPath = new Path(path); + if (!fs.exists(newDirectoryPath)) { + // Create new Directory + fs.mkdirs(newDirectoryPath); + LOGGER.debug("Path {} created.", path); + } + + String dir = System.getProperty("user.dir") + "/src/test/java/com/teragrep/cfe_39/mockHdfsFiles"; + Set listOfFiles = Stream + .of(Objects.requireNonNull(new File(dir).listFiles())) + .filter(file -> !file.isDirectory()) + .map(File::getName) + .collect(Collectors.toSet()); + // Loop through all the avro files + for (String fileName : listOfFiles) { + String pathname = dir + "/" + fileName; + File avroFile = new File(pathname); + //==== Write file + LOGGER.debug("Begin Write file into hdfs"); + //Create a path + Path hdfswritepath = new Path(newDirectoryPath + "/" + avroFile.getName()); // filename should be set according to the requirements: 0.12345 where 0 is Kafka partition and 12345 is Kafka offset. + Assertions.assertFalse(fs.exists(hdfswritepath)); + Path readPath = new Path(avroFile.getPath()); + fs.copyFromLocalFile(readPath, hdfswritepath); + // Set fileTimestampA/fileTimestampB to something like 157784760000 to trigger pruning, -1 to not alter the timestamp. + if (Objects.equals(hdfswritepath.toString(), "hdfs:/opt/teragrep/cfe_39/srv/testConsumerTopic/0.9")) { + fs.setTimes(hdfswritepath, fileTimestampA, -1); + } + else if ( + Objects.equals(hdfswritepath.toString(), "hdfs:/opt/teragrep/cfe_39/srv/testConsumerTopic/0.13") + ) { + fs.setTimes(hdfswritepath, fileTimestampB, -1); + } + LOGGER.debug("End Write file into hdfs"); + LOGGER.debug("\nFile committed to HDFS, file writepath should be: {}\n", hdfswritepath); + } + }); + } +} From 58b53d87911e6b92b9f660c2aaf59ded2b141e1a Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Tue, 18 Jun 2024 11:29:00 +0300 Subject: [PATCH 098/146] Removed helper functions and if-statement usage from Pruning tests. --- .../cfe_39/PruningOneNewFileTest.java | 65 +++++++--------- .../cfe_39/PruningOneOldFileTest.java | 65 +++++++--------- .../cfe_39/PruningOneOldOneNewFileTest.java | 77 ++++++++----------- .../cfe_39/PruningTwoNewFilesTest.java | 77 ++++++++----------- .../cfe_39/PruningTwoOldFilesTest.java | 76 +++++++----------- 5 files changed, 143 insertions(+), 217 deletions(-) diff --git a/src/test/java/com/teragrep/cfe_39/PruningOneNewFileTest.java b/src/test/java/com/teragrep/cfe_39/PruningOneNewFileTest.java index 0dcb0b25..a4e40abb 100644 --- a/src/test/java/com/teragrep/cfe_39/PruningOneNewFileTest.java +++ b/src/test/java/com/teragrep/cfe_39/PruningOneNewFileTest.java @@ -103,7 +103,33 @@ public void startMiniCluster() { System.setProperty("HADOOP_USER_NAME", "hdfs"); System.setProperty("hadoop.home.dir", "/"); fs = FileSystem.get(URI.create(hdfsURI), fsConf); - insertMockFile(-1); // Insert 1 new mock file. + // Inserts a single pre-made avro-file with a new timestamp to HDFS, which is normally generated during data ingestion from mock kafka consumer. + String path = config.getHdfsPath() + "/" + "testConsumerTopic"; // "hdfs:///opt/teragrep/cfe_39/srv/testConsumerTopic" + // Sets the directory where the data should be stored, if the directory doesn't exist then it's created. + Path newDirectoryPath = new Path(path); + // Create new Directory + fs.mkdirs(newDirectoryPath); + LOGGER.debug("Path {} created.", path); + String dir = System.getProperty("user.dir") + "/src/test/java/com/teragrep/cfe_39/mockHdfsFiles"; + Set listOfFiles = Stream + .of(Objects.requireNonNull(new File(dir).listFiles())) + .filter(file -> !file.isDirectory()) + .map(File::getName) + .collect(Collectors.toSet()); + String fileName = "0.9"; + Assertions.assertTrue(listOfFiles.contains(fileName)); + String pathname = dir + "/" + fileName; + File avroFile = new File(pathname); + //==== Write file + LOGGER.debug("Begin Write file into hdfs"); + //Create a path + Path hdfswritepath = new Path(newDirectoryPath + "/" + avroFile.getName()); // filename should be set according to the requirements: 0.12345 where 0 is Kafka partition and 12345 is Kafka offset. + Assertions.assertFalse(fs.exists(hdfswritepath)); + Path readPath = new Path(avroFile.getPath()); + fs.copyFromLocalFile(readPath, hdfswritepath); + fs.setTimes(hdfswritepath, -1, -1); + LOGGER.debug("End Write file into hdfs"); + LOGGER.debug("\nFile committed to HDFS, file writepath should be: {}\n", hdfswritepath); }); } @@ -136,41 +162,4 @@ public void oneNewFileTest() { Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); }); } - - // Inserts a single pre-made avro-file to HDFS, which is normally generated during data ingestion from mock kafka consumer. - private void insertMockFile(long fileTimestamp) { - String path = config.getHdfsPath() + "/" + "testConsumerTopic"; // "hdfs:///opt/teragrep/cfe_39/srv/testConsumerTopic" - assertDoesNotThrow(() -> { - - // Sets the directory where the data should be stored, if the directory doesn't exist then it's created. - Path newDirectoryPath = new Path(path); - if (!fs.exists(newDirectoryPath)) { - // Create new Directory - fs.mkdirs(newDirectoryPath); - LOGGER.debug("Path {} created.", path); - } - - String dir = System.getProperty("user.dir") + "/src/test/java/com/teragrep/cfe_39/mockHdfsFiles"; - Set listOfFiles = Stream - .of(Objects.requireNonNull(new File(dir).listFiles())) - .filter(file -> !file.isDirectory()) - .map(File::getName) - .collect(Collectors.toSet()); - String fileName = "0.9"; - Assertions.assertTrue(listOfFiles.contains(fileName)); - String pathname = dir + "/" + fileName; - File avroFile = new File(pathname); - //==== Write file - LOGGER.debug("Begin Write file into hdfs"); - //Create a path - Path hdfswritepath = new Path(newDirectoryPath + "/" + avroFile.getName()); // filename should be set according to the requirements: 0.12345 where 0 is Kafka partition and 12345 is Kafka offset. - Assertions.assertFalse(fs.exists(hdfswritepath)); - Path readPath = new Path(avroFile.getPath()); - fs.copyFromLocalFile(readPath, hdfswritepath); - // Set fileTimestamp to something like 157784760000 to trigger pruning, -1 to not alter the timestamp. - fs.setTimes(hdfswritepath, fileTimestamp, -1); - LOGGER.debug("End Write file into hdfs"); - LOGGER.debug("\nFile committed to HDFS, file writepath should be: {}\n", hdfswritepath); - }); - } } diff --git a/src/test/java/com/teragrep/cfe_39/PruningOneOldFileTest.java b/src/test/java/com/teragrep/cfe_39/PruningOneOldFileTest.java index a62140cf..516523c8 100644 --- a/src/test/java/com/teragrep/cfe_39/PruningOneOldFileTest.java +++ b/src/test/java/com/teragrep/cfe_39/PruningOneOldFileTest.java @@ -103,7 +103,33 @@ public void startMiniCluster() { System.setProperty("HADOOP_USER_NAME", "hdfs"); System.setProperty("hadoop.home.dir", "/"); fs = FileSystem.get(URI.create(hdfsURI), fsConf); - insertMockFile(157784760000L); // Insert 1 old mock file. + // Inserts a single pre-made avro-file with an olf timestamp to HDFS, which is normally generated during data ingestion from mock kafka consumer. + String path = config.getHdfsPath() + "/" + "testConsumerTopic"; // "hdfs:///opt/teragrep/cfe_39/srv/testConsumerTopic" + // Sets the directory where the data should be stored, if the directory doesn't exist then it's created. + Path newDirectoryPath = new Path(path); + // Create new Directory + fs.mkdirs(newDirectoryPath); + LOGGER.debug("Path {} created.", path); + String dir = System.getProperty("user.dir") + "/src/test/java/com/teragrep/cfe_39/mockHdfsFiles"; + Set listOfFiles = Stream + .of(Objects.requireNonNull(new File(dir).listFiles())) + .filter(file -> !file.isDirectory()) + .map(File::getName) + .collect(Collectors.toSet()); + String fileName = "0.9"; + Assertions.assertTrue(listOfFiles.contains(fileName)); + String pathname = dir + "/" + fileName; + File avroFile = new File(pathname); + //==== Write file + LOGGER.debug("Begin Write file into hdfs"); + //Create a path + Path hdfswritepath = new Path(newDirectoryPath + "/" + avroFile.getName()); // filename should be set according to the requirements: 0.12345 where 0 is Kafka partition and 12345 is Kafka offset. + Assertions.assertFalse(fs.exists(hdfswritepath)); + Path readPath = new Path(avroFile.getPath()); + fs.copyFromLocalFile(readPath, hdfswritepath); + fs.setTimes(hdfswritepath, 157784760000L, -1); + LOGGER.debug("End Write file into hdfs"); + LOGGER.debug("\nFile committed to HDFS, file writepath should be: {}\n", hdfswritepath); }); } @@ -136,41 +162,4 @@ public void oneOldFileTest() { Assertions.assertFalse(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); }); } - - // Inserts a single pre-made avro-file to HDFS, which is normally generated during data ingestion from mock kafka consumer. - private void insertMockFile(long fileTimestamp) { - String path = config.getHdfsPath() + "/" + "testConsumerTopic"; // "hdfs:///opt/teragrep/cfe_39/srv/testConsumerTopic" - assertDoesNotThrow(() -> { - - // Sets the directory where the data should be stored, if the directory doesn't exist then it's created. - Path newDirectoryPath = new Path(path); - if (!fs.exists(newDirectoryPath)) { - // Create new Directory - fs.mkdirs(newDirectoryPath); - LOGGER.debug("Path {} created.", path); - } - - String dir = System.getProperty("user.dir") + "/src/test/java/com/teragrep/cfe_39/mockHdfsFiles"; - Set listOfFiles = Stream - .of(Objects.requireNonNull(new File(dir).listFiles())) - .filter(file -> !file.isDirectory()) - .map(File::getName) - .collect(Collectors.toSet()); - String fileName = "0.9"; - Assertions.assertTrue(listOfFiles.contains(fileName)); - String pathname = dir + "/" + fileName; - File avroFile = new File(pathname); - //==== Write file - LOGGER.debug("Begin Write file into hdfs"); - //Create a path - Path hdfswritepath = new Path(newDirectoryPath + "/" + avroFile.getName()); // filename should be set according to the requirements: 0.12345 where 0 is Kafka partition and 12345 is Kafka offset. - Assertions.assertFalse(fs.exists(hdfswritepath)); - Path readPath = new Path(avroFile.getPath()); - fs.copyFromLocalFile(readPath, hdfswritepath); - // Set fileTimestamp to something like 157784760000 to trigger pruning, -1 to not alter the timestamp. - fs.setTimes(hdfswritepath, fileTimestamp, -1); - LOGGER.debug("End Write file into hdfs"); - LOGGER.debug("\nFile committed to HDFS, file writepath should be: {}\n", hdfswritepath); - }); - } } diff --git a/src/test/java/com/teragrep/cfe_39/PruningOneOldOneNewFileTest.java b/src/test/java/com/teragrep/cfe_39/PruningOneOldOneNewFileTest.java index 655e0852..1fca4dd4 100644 --- a/src/test/java/com/teragrep/cfe_39/PruningOneOldOneNewFileTest.java +++ b/src/test/java/com/teragrep/cfe_39/PruningOneOldOneNewFileTest.java @@ -103,7 +103,36 @@ public void startMiniCluster() { System.setProperty("HADOOP_USER_NAME", "hdfs"); System.setProperty("hadoop.home.dir", "/"); fs = FileSystem.get(URI.create(hdfsURI), fsConf); - insertMockFiles(157784760000L, -1); // Insert 2 mock files, one with old timestamp and one with new timestamp. + /* Inserts pre-made avro-files to HDFS, which are normally generated during data ingestion from mock kafka consumer. + One file has new timestamp and another old timestamp.*/ + String path = config.getHdfsPath() + "/" + "testConsumerTopic"; // "hdfs:///opt/teragrep/cfe_39/srv/testConsumerTopic" + // Sets the directory where the data should be stored, if the directory doesn't exist then it's created. + Path newDirectoryPath = new Path(path); + // Create new Directory + fs.mkdirs(newDirectoryPath); + LOGGER.debug("Path {} created.", path); + String dir = System.getProperty("user.dir") + "/src/test/java/com/teragrep/cfe_39/mockHdfsFiles"; + Set listOfFiles = Stream + .of(Objects.requireNonNull(new File(dir).listFiles())) + .filter(file -> !file.isDirectory()) + .map(File::getName) + .collect(Collectors.toSet()); + // Loop through all the avro files + for (String fileName : listOfFiles) { + String pathname = dir + "/" + fileName; + File avroFile = new File(pathname); + //==== Write file + LOGGER.debug("Begin Write file into hdfs"); + //Create a path + Path hdfswritepath = new Path(newDirectoryPath + "/" + avroFile.getName()); // filename should be set according to the requirements: 0.12345 where 0 is Kafka partition and 12345 is Kafka offset. + Assertions.assertFalse(fs.exists(hdfswritepath)); + Path readPath = new Path(avroFile.getPath()); + fs.copyFromLocalFile(readPath, hdfswritepath); + LOGGER.debug("End Write file into hdfs"); + LOGGER.debug("\nFile committed to HDFS, file writepath should be: {}\n", hdfswritepath); + } + fs.setTimes(new Path("hdfs:/opt/teragrep/cfe_39/srv/testConsumerTopic/0.9"), 157784760000L, -1); + fs.setTimes(new Path("hdfs:/opt/teragrep/cfe_39/srv/testConsumerTopic/0.13"), -1, -1); }); } @@ -138,50 +167,4 @@ public void oneOldOneNewFileTest() { Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.13"))); }); } - - // Inserts pre-made avro-files to HDFS, which are normally generated during data ingestion from mock kafka consumer. - private void insertMockFiles(long fileTimestampA, long fileTimestampB) { - String path = config.getHdfsPath() + "/" + "testConsumerTopic"; // "hdfs:///opt/teragrep/cfe_39/srv/testConsumerTopic" - //Get the filesystem - HDFS - assertDoesNotThrow(() -> { - - // Sets the directory where the data should be stored, if the directory doesn't exist then it's created. - Path newDirectoryPath = new Path(path); - if (!fs.exists(newDirectoryPath)) { - // Create new Directory - fs.mkdirs(newDirectoryPath); - LOGGER.debug("Path {} created.", path); - } - - String dir = System.getProperty("user.dir") + "/src/test/java/com/teragrep/cfe_39/mockHdfsFiles"; - Set listOfFiles = Stream - .of(Objects.requireNonNull(new File(dir).listFiles())) - .filter(file -> !file.isDirectory()) - .map(File::getName) - .collect(Collectors.toSet()); - // Loop through all the avro files - for (String fileName : listOfFiles) { - String pathname = dir + "/" + fileName; - File avroFile = new File(pathname); - //==== Write file - LOGGER.debug("Begin Write file into hdfs"); - //Create a path - Path hdfswritepath = new Path(newDirectoryPath + "/" + avroFile.getName()); // filename should be set according to the requirements: 0.12345 where 0 is Kafka partition and 12345 is Kafka offset. - Assertions.assertFalse(fs.exists(hdfswritepath)); - Path readPath = new Path(avroFile.getPath()); - fs.copyFromLocalFile(readPath, hdfswritepath); - // Set fileTimestampA/fileTimestampB to something like 157784760000 to trigger pruning, -1 to not alter the timestamp. - if (Objects.equals(hdfswritepath.toString(), "hdfs:/opt/teragrep/cfe_39/srv/testConsumerTopic/0.9")) { - fs.setTimes(hdfswritepath, fileTimestampA, -1); - } - else if ( - Objects.equals(hdfswritepath.toString(), "hdfs:/opt/teragrep/cfe_39/srv/testConsumerTopic/0.13") - ) { - fs.setTimes(hdfswritepath, fileTimestampB, -1); - } - LOGGER.debug("End Write file into hdfs"); - LOGGER.debug("\nFile committed to HDFS, file writepath should be: {}\n", hdfswritepath); - } - }); - } } diff --git a/src/test/java/com/teragrep/cfe_39/PruningTwoNewFilesTest.java b/src/test/java/com/teragrep/cfe_39/PruningTwoNewFilesTest.java index 1d66875f..06e47f92 100644 --- a/src/test/java/com/teragrep/cfe_39/PruningTwoNewFilesTest.java +++ b/src/test/java/com/teragrep/cfe_39/PruningTwoNewFilesTest.java @@ -103,7 +103,36 @@ public void startMiniCluster() { System.setProperty("HADOOP_USER_NAME", "hdfs"); System.setProperty("hadoop.home.dir", "/"); fs = FileSystem.get(URI.create(hdfsURI), fsConf); - insertMockFiles(-1, -1); // Insert 2 mock files with new timestamps so pruning should not trigger on them. + + // Inserts pre-made avro-files with new timestamps to HDFS, which are normally generated during data ingestion from mock kafka consumer. + + String path = config.getHdfsPath() + "/" + "testConsumerTopic"; // "hdfs:///opt/teragrep/cfe_39/srv/testConsumerTopic" + // Sets the directory where the data should be stored, if the directory doesn't exist then it's created. + Path newDirectoryPath = new Path(path); + // Create new Directory + fs.mkdirs(newDirectoryPath); + LOGGER.debug("Path {} created.", path); + + String dir = System.getProperty("user.dir") + "/src/test/java/com/teragrep/cfe_39/mockHdfsFiles"; + Set listOfFiles = Stream + .of(Objects.requireNonNull(new File(dir).listFiles())) + .filter(file -> !file.isDirectory()) + .map(File::getName) + .collect(Collectors.toSet()); + // Loop through all the avro files + for (String fileName : listOfFiles) { + String pathname = dir + "/" + fileName; + File avroFile = new File(pathname); + //==== Write file + LOGGER.debug("Begin Write file into hdfs"); + //Create a path + Path hdfswritepath = new Path(newDirectoryPath + "/" + avroFile.getName()); // filename should be set according to the requirements: 0.12345 where 0 is Kafka partition and 12345 is Kafka offset. + Assertions.assertFalse(fs.exists(hdfswritepath)); + Path readPath = new Path(avroFile.getPath()); + fs.copyFromLocalFile(readPath, hdfswritepath); + LOGGER.debug("End Write file into hdfs"); + LOGGER.debug("\nFile committed to HDFS, file writepath should be: {}\n", hdfswritepath); + } }); } @@ -136,50 +165,4 @@ public void twoNewFilesTest() { Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.13"))); }); } - - // Inserts pre-made avro-files to HDFS, which are normally generated during data ingestion from mock kafka consumer. - private void insertMockFiles(long fileTimestampA, long fileTimestampB) { - String path = config.getHdfsPath() + "/" + "testConsumerTopic"; // "hdfs:///opt/teragrep/cfe_39/srv/testConsumerTopic" - //Get the filesystem - HDFS - assertDoesNotThrow(() -> { - - // Sets the directory where the data should be stored, if the directory doesn't exist then it's created. - Path newDirectoryPath = new Path(path); - if (!fs.exists(newDirectoryPath)) { - // Create new Directory - fs.mkdirs(newDirectoryPath); - LOGGER.debug("Path {} created.", path); - } - - String dir = System.getProperty("user.dir") + "/src/test/java/com/teragrep/cfe_39/mockHdfsFiles"; - Set listOfFiles = Stream - .of(Objects.requireNonNull(new File(dir).listFiles())) - .filter(file -> !file.isDirectory()) - .map(File::getName) - .collect(Collectors.toSet()); - // Loop through all the avro files - for (String fileName : listOfFiles) { - String pathname = dir + "/" + fileName; - File avroFile = new File(pathname); - //==== Write file - LOGGER.debug("Begin Write file into hdfs"); - //Create a path - Path hdfswritepath = new Path(newDirectoryPath + "/" + avroFile.getName()); // filename should be set according to the requirements: 0.12345 where 0 is Kafka partition and 12345 is Kafka offset. - Assertions.assertFalse(fs.exists(hdfswritepath)); - Path readPath = new Path(avroFile.getPath()); - fs.copyFromLocalFile(readPath, hdfswritepath); - // Set fileTimestampA/fileTimestampB to something like 157784760000 to trigger pruning, -1 to not alter the timestamp. - if (Objects.equals(hdfswritepath.toString(), "hdfs:/opt/teragrep/cfe_39/srv/testConsumerTopic/0.9")) { - fs.setTimes(hdfswritepath, fileTimestampA, -1); - } - else if ( - Objects.equals(hdfswritepath.toString(), "hdfs:/opt/teragrep/cfe_39/srv/testConsumerTopic/0.13") - ) { - fs.setTimes(hdfswritepath, fileTimestampB, -1); - } - LOGGER.debug("End Write file into hdfs"); - LOGGER.debug("\nFile committed to HDFS, file writepath should be: {}\n", hdfswritepath); - } - }); - } } diff --git a/src/test/java/com/teragrep/cfe_39/PruningTwoOldFilesTest.java b/src/test/java/com/teragrep/cfe_39/PruningTwoOldFilesTest.java index 00dfb299..eb4804ea 100644 --- a/src/test/java/com/teragrep/cfe_39/PruningTwoOldFilesTest.java +++ b/src/test/java/com/teragrep/cfe_39/PruningTwoOldFilesTest.java @@ -103,7 +103,35 @@ public void startMiniCluster() { System.setProperty("HADOOP_USER_NAME", "hdfs"); System.setProperty("hadoop.home.dir", "/"); fs = FileSystem.get(URI.create(hdfsURI), fsConf); - insertMockFiles(157784760000L, 157784760000L); // Insert 2 mock files with old timestamps so pruning should trigger on them. + // Inserts pre-made avro-files with old timestamps to HDFS, which are normally generated during data ingestion from mock kafka consumer. + + String path = config.getHdfsPath() + "/" + "testConsumerTopic"; // "hdfs:///opt/teragrep/cfe_39/srv/testConsumerTopic" + // Sets the directory where the data should be stored, if the directory doesn't exist then it's created. + Path newDirectoryPath = new Path(path); + // Create new Directory + fs.mkdirs(newDirectoryPath); + LOGGER.debug("Path {} created.", path); + String dir = System.getProperty("user.dir") + "/src/test/java/com/teragrep/cfe_39/mockHdfsFiles"; + Set listOfFiles = Stream + .of(Objects.requireNonNull(new File(dir).listFiles())) + .filter(file -> !file.isDirectory()) + .map(File::getName) + .collect(Collectors.toSet()); + // Loop through all the avro files + for (String fileName : listOfFiles) { + String pathname = dir + "/" + fileName; + File avroFile = new File(pathname); + //==== Write file + LOGGER.debug("Begin Write file into hdfs"); + //Create a path + Path hdfswritepath = new Path(newDirectoryPath + "/" + avroFile.getName()); // filename should be set according to the requirements: 0.12345 where 0 is Kafka partition and 12345 is Kafka offset. + Assertions.assertFalse(fs.exists(hdfswritepath)); + Path readPath = new Path(avroFile.getPath()); + fs.copyFromLocalFile(readPath, hdfswritepath); + fs.setTimes(hdfswritepath, 157784760000L, -1); + LOGGER.debug("End Write file into hdfs"); + LOGGER.debug("\nFile committed to HDFS, file writepath should be: {}\n", hdfswritepath); + } }); } @@ -139,50 +167,4 @@ public void twoOldFilesTest() { .assertFalse(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.13"))); }); } - - // Inserts pre-made avro-files to HDFS, which are normally generated during data ingestion from mock kafka consumer. - private void insertMockFiles(long fileTimestampA, long fileTimestampB) { - String path = config.getHdfsPath() + "/" + "testConsumerTopic"; // "hdfs:///opt/teragrep/cfe_39/srv/testConsumerTopic" - //Get the filesystem - HDFS - assertDoesNotThrow(() -> { - - // Sets the directory where the data should be stored, if the directory doesn't exist then it's created. - Path newDirectoryPath = new Path(path); - if (!fs.exists(newDirectoryPath)) { - // Create new Directory - fs.mkdirs(newDirectoryPath); - LOGGER.debug("Path {} created.", path); - } - - String dir = System.getProperty("user.dir") + "/src/test/java/com/teragrep/cfe_39/mockHdfsFiles"; - Set listOfFiles = Stream - .of(Objects.requireNonNull(new File(dir).listFiles())) - .filter(file -> !file.isDirectory()) - .map(File::getName) - .collect(Collectors.toSet()); - // Loop through all the avro files - for (String fileName : listOfFiles) { - String pathname = dir + "/" + fileName; - File avroFile = new File(pathname); - //==== Write file - LOGGER.debug("Begin Write file into hdfs"); - //Create a path - Path hdfswritepath = new Path(newDirectoryPath + "/" + avroFile.getName()); // filename should be set according to the requirements: 0.12345 where 0 is Kafka partition and 12345 is Kafka offset. - Assertions.assertFalse(fs.exists(hdfswritepath)); - Path readPath = new Path(avroFile.getPath()); - fs.copyFromLocalFile(readPath, hdfswritepath); - // Set fileTimestampA/fileTimestampB to something like 157784760000 to trigger pruning, -1 to not alter the timestamp. - if (Objects.equals(hdfswritepath.toString(), "hdfs:/opt/teragrep/cfe_39/srv/testConsumerTopic/0.9")) { - fs.setTimes(hdfswritepath, fileTimestampA, -1); - } - else if ( - Objects.equals(hdfswritepath.toString(), "hdfs:/opt/teragrep/cfe_39/srv/testConsumerTopic/0.13") - ) { - fs.setTimes(hdfswritepath, fileTimestampB, -1); - } - LOGGER.debug("End Write file into hdfs"); - LOGGER.debug("\nFile committed to HDFS, file writepath should be: {}\n", hdfswritepath); - } - }); - } } From 5c40bb9ad57306901e7013967b235f3f22878eaa Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Tue, 18 Jun 2024 12:02:24 +0300 Subject: [PATCH 099/146] Removed helper functions and if-statement usage from ingestion tests. --- .../cfe_39/Ingestion1Old1NewFileTest.java | 76 +++++++----------- .../cfe_39/Ingestion2NewFilesTest.java | 74 +++++++----------- .../cfe_39/Ingestion2OldFilesTest.java | 77 ++++++++----------- 3 files changed, 86 insertions(+), 141 deletions(-) diff --git a/src/test/java/com/teragrep/cfe_39/Ingestion1Old1NewFileTest.java b/src/test/java/com/teragrep/cfe_39/Ingestion1Old1NewFileTest.java index 0d5f0fa9..c588905b 100644 --- a/src/test/java/com/teragrep/cfe_39/Ingestion1Old1NewFileTest.java +++ b/src/test/java/com/teragrep/cfe_39/Ingestion1Old1NewFileTest.java @@ -103,7 +103,35 @@ public void startMiniCluster() { System.setProperty("HADOOP_USER_NAME", "hdfs"); System.setProperty("hadoop.home.dir", "/"); fs = FileSystem.get(URI.create(hdfsURI), fsConf); - insertMockFiles(157784760000L, -1); // Insert 2 mock files (0.9 and 0.13) with old timestamp on 0.9 and new timestamp on 0.13. + // Inserts pre-made avro-files to HDFS where one file has new timestamp and other old, which are normally generated during data ingestion from mock kafka consumer. + String path = config.getHdfsPath() + "/" + "testConsumerTopic"; // "hdfs:///opt/teragrep/cfe_39/srv/testConsumerTopic" + // Sets the directory where the data should be stored, if the directory doesn't exist then it's created. + Path newDirectoryPath = new Path(path); + // Create new Directory + fs.mkdirs(newDirectoryPath); + LOGGER.debug("Path {} created.", path); + String dir = System.getProperty("user.dir") + "/src/test/java/com/teragrep/cfe_39/mockHdfsFiles"; + Set listOfFiles = Stream + .of(Objects.requireNonNull(new File(dir).listFiles())) + .filter(file -> !file.isDirectory()) + .map(File::getName) + .collect(Collectors.toSet()); + // Loop through all the avro files + for (String fileName : listOfFiles) { + String pathname = dir + "/" + fileName; + File avroFile = new File(pathname); + //==== Write file + LOGGER.debug("Begin Write file into hdfs"); + //Create a path + Path hdfswritepath = new Path(newDirectoryPath + "/" + avroFile.getName()); // filename should be set according to the requirements: 0.12345 where 0 is Kafka partition and 12345 is Kafka offset. + Assertions.assertFalse(fs.exists(hdfswritepath)); + Path readPath = new Path(avroFile.getPath()); + fs.copyFromLocalFile(readPath, hdfswritepath); + LOGGER.debug("End Write file into hdfs"); + LOGGER.debug("\nFile committed to HDFS, file writepath should be: {}\n", hdfswritepath); + } + fs.setTimes(new Path("hdfs:/opt/teragrep/cfe_39/srv/testConsumerTopic/0.9"), 157784760000L, -1); + fs.setTimes(new Path("hdfs:/opt/teragrep/cfe_39/srv/testConsumerTopic/0.13"), -1, -1); }); } @@ -168,50 +196,4 @@ public void ingestion1Old1NewFileTest() { Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "9.13"))); }); } - - // Inserts pre-made avro-files to HDFS, which are normally generated during data ingestion from mock kafka consumer. - private void insertMockFiles(long fileTimestampA, long fileTimestampB) { - String path = config.getHdfsPath() + "/" + "testConsumerTopic"; // "hdfs:///opt/teragrep/cfe_39/srv/testConsumerTopic" - //Get the filesystem - HDFS - assertDoesNotThrow(() -> { - - // Sets the directory where the data should be stored, if the directory doesn't exist then it's created. - Path newDirectoryPath = new Path(path); - if (!fs.exists(newDirectoryPath)) { - // Create new Directory - fs.mkdirs(newDirectoryPath); - LOGGER.debug("Path {} created.", path); - } - - String dir = System.getProperty("user.dir") + "/src/test/java/com/teragrep/cfe_39/mockHdfsFiles"; - Set listOfFiles = Stream - .of(Objects.requireNonNull(new File(dir).listFiles())) - .filter(file -> !file.isDirectory()) - .map(File::getName) - .collect(Collectors.toSet()); - // Loop through all the avro files - for (String fileName : listOfFiles) { - String pathname = dir + "/" + fileName; - File avroFile = new File(pathname); - //==== Write file - LOGGER.debug("Begin Write file into hdfs"); - //Create a path - Path hdfswritepath = new Path(newDirectoryPath + "/" + avroFile.getName()); // filename should be set according to the requirements: 0.12345 where 0 is Kafka partition and 12345 is Kafka offset. - Assertions.assertFalse(fs.exists(hdfswritepath)); - Path readPath = new Path(avroFile.getPath()); - fs.copyFromLocalFile(readPath, hdfswritepath); - // Set fileTimestampA/fileTimestampB to something like 157784760000 to trigger pruning, -1 to not alter the timestamp. - if (Objects.equals(hdfswritepath.toString(), "hdfs:/opt/teragrep/cfe_39/srv/testConsumerTopic/0.9")) { - fs.setTimes(hdfswritepath, fileTimestampA, -1); - } - else if ( - Objects.equals(hdfswritepath.toString(), "hdfs:/opt/teragrep/cfe_39/srv/testConsumerTopic/0.13") - ) { - fs.setTimes(hdfswritepath, fileTimestampB, -1); - } - LOGGER.debug("End Write file into hdfs"); - LOGGER.debug("\nFile committed to HDFS, file writepath should be: {}\n", hdfswritepath); - } - }); - } } diff --git a/src/test/java/com/teragrep/cfe_39/Ingestion2NewFilesTest.java b/src/test/java/com/teragrep/cfe_39/Ingestion2NewFilesTest.java index f8eb2aee..3c04dddd 100644 --- a/src/test/java/com/teragrep/cfe_39/Ingestion2NewFilesTest.java +++ b/src/test/java/com/teragrep/cfe_39/Ingestion2NewFilesTest.java @@ -106,7 +106,33 @@ public void startMiniCluster() { System.setProperty("HADOOP_USER_NAME", "hdfs"); System.setProperty("hadoop.home.dir", "/"); fs = FileSystem.get(URI.create(hdfsURI), fsConf); - insertMockFiles(-1, -1); // Insert 2 mock files (0.9 and 0.13) with new timestamps so pruning should not trigger on them. + // Inserts pre-made avro-files with new timestamps to HDFS, which are normally generated during data ingestion from mock kafka consumer. + String path = config.getHdfsPath() + "/" + "testConsumerTopic"; // "hdfs:///opt/teragrep/cfe_39/srv/testConsumerTopic" + // Sets the directory where the data should be stored, if the directory doesn't exist then it's created. + Path newDirectoryPath = new Path(path); + // Create new Directory + fs.mkdirs(newDirectoryPath); + LOGGER.debug("Path {} created.", path); + String dir = System.getProperty("user.dir") + "/src/test/java/com/teragrep/cfe_39/mockHdfsFiles"; + Set listOfFiles = Stream + .of(Objects.requireNonNull(new File(dir).listFiles())) + .filter(file -> !file.isDirectory()) + .map(File::getName) + .collect(Collectors.toSet()); + // Loop through all the avro files + for (String fileName : listOfFiles) { + String pathname = dir + "/" + fileName; + File avroFile = new File(pathname); + //==== Write file + LOGGER.debug("Begin Write file into hdfs"); + //Create a path + Path hdfswritepath = new Path(newDirectoryPath + "/" + avroFile.getName()); // filename should be set according to the requirements: 0.12345 where 0 is Kafka partition and 12345 is Kafka offset. + Assertions.assertFalse(fs.exists(hdfswritepath)); + Path readPath = new Path(avroFile.getPath()); + fs.copyFromLocalFile(readPath, hdfswritepath); + LOGGER.debug("End Write file into hdfs"); + LOGGER.debug("\nFile committed to HDFS, file writepath should be: {}\n", hdfswritepath); + } }); } @@ -589,50 +615,4 @@ record = reader.next(record); Assertions.assertEquals(10, partitionCounter); }); } - - // Inserts pre-made avro-files to HDFS, which are normally generated during data ingestion from mock kafka consumer. - private void insertMockFiles(long fileTimestampA, long fileTimestampB) { - String path = config.getHdfsPath() + "/" + "testConsumerTopic"; // "hdfs:///opt/teragrep/cfe_39/srv/testConsumerTopic" - //Get the filesystem - HDFS - assertDoesNotThrow(() -> { - - // Sets the directory where the data should be stored, if the directory doesn't exist then it's created. - Path newDirectoryPath = new Path(path); - if (!fs.exists(newDirectoryPath)) { - // Create new Directory - fs.mkdirs(newDirectoryPath); - LOGGER.debug("Path {} created.", path); - } - - String dir = System.getProperty("user.dir") + "/src/test/java/com/teragrep/cfe_39/mockHdfsFiles"; - Set listOfFiles = Stream - .of(Objects.requireNonNull(new File(dir).listFiles())) - .filter(file -> !file.isDirectory()) - .map(File::getName) - .collect(Collectors.toSet()); - // Loop through all the avro files - for (String fileName : listOfFiles) { - String pathname = dir + "/" + fileName; - File avroFile = new File(pathname); - //==== Write file - LOGGER.debug("Begin Write file into hdfs"); - //Create a path - Path hdfswritepath = new Path(newDirectoryPath + "/" + avroFile.getName()); // filename should be set according to the requirements: 0.12345 where 0 is Kafka partition and 12345 is Kafka offset. - Assertions.assertFalse(fs.exists(hdfswritepath)); - Path readPath = new Path(avroFile.getPath()); - fs.copyFromLocalFile(readPath, hdfswritepath); - // Set fileTimestampA/fileTimestampB to something like 157784760000 to trigger pruning, -1 to not alter the timestamp. - if (Objects.equals(hdfswritepath.toString(), "hdfs:/opt/teragrep/cfe_39/srv/testConsumerTopic/0.9")) { - fs.setTimes(hdfswritepath, fileTimestampA, -1); - } - else if ( - Objects.equals(hdfswritepath.toString(), "hdfs:/opt/teragrep/cfe_39/srv/testConsumerTopic/0.13") - ) { - fs.setTimes(hdfswritepath, fileTimestampB, -1); - } - LOGGER.debug("End Write file into hdfs"); - LOGGER.debug("\nFile committed to HDFS, file writepath should be: {}\n", hdfswritepath); - } - }); - } } diff --git a/src/test/java/com/teragrep/cfe_39/Ingestion2OldFilesTest.java b/src/test/java/com/teragrep/cfe_39/Ingestion2OldFilesTest.java index 3f55c2fa..9b84670b 100644 --- a/src/test/java/com/teragrep/cfe_39/Ingestion2OldFilesTest.java +++ b/src/test/java/com/teragrep/cfe_39/Ingestion2OldFilesTest.java @@ -103,7 +103,36 @@ public void startMiniCluster() { System.setProperty("HADOOP_USER_NAME", "hdfs"); System.setProperty("hadoop.home.dir", "/"); fs = FileSystem.get(URI.create(hdfsURI), fsConf); - insertMockFiles(157784760000L, 157784760000L); // Insert 2 mock files (0.9 and 0.13) with old timestamps so pruning should trigger on them. + // Inserts pre-made avro-files with old timestamps to HDFS, which are normally generated during data ingestion from mock kafka consumer. + String path = config.getHdfsPath() + "/" + "testConsumerTopic"; // "hdfs:///opt/teragrep/cfe_39/srv/testConsumerTopic" + // Sets the directory where the data should be stored, if the directory doesn't exist then it's created. + Path newDirectoryPath = new Path(path); + // Create new Directory + fs.mkdirs(newDirectoryPath); + LOGGER.debug("Path {} created.", path); + + String dir = System.getProperty("user.dir") + "/src/test/java/com/teragrep/cfe_39/mockHdfsFiles"; + Set listOfFiles = Stream + .of(Objects.requireNonNull(new File(dir).listFiles())) + .filter(file -> !file.isDirectory()) + .map(File::getName) + .collect(Collectors.toSet()); + // Loop through all the avro files + for (String fileName : listOfFiles) { + String pathname = dir + "/" + fileName; + File avroFile = new File(pathname); + //==== Write file + LOGGER.debug("Begin Write file into hdfs"); + //Create a path + Path hdfswritepath = new Path(newDirectoryPath + "/" + avroFile.getName()); // filename should be set according to the requirements: 0.12345 where 0 is Kafka partition and 12345 is Kafka offset. + Assertions.assertFalse(fs.exists(hdfswritepath)); + Path readPath = new Path(avroFile.getPath()); + fs.copyFromLocalFile(readPath, hdfswritepath); + LOGGER.debug("End Write file into hdfs"); + LOGGER.debug("\nFile committed to HDFS, file writepath should be: {}\n", hdfswritepath); + } + fs.setTimes(new Path("hdfs:/opt/teragrep/cfe_39/srv/testConsumerTopic/0.9"), 157784760000L, -1); + fs.setTimes(new Path("hdfs:/opt/teragrep/cfe_39/srv/testConsumerTopic/0.13"), 157784760000L, -1); }); } @@ -169,50 +198,4 @@ public void ingestion2OldFilesTest() { Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "9.13"))); }); } - - // Inserts pre-made avro-files to HDFS, which are normally generated during data ingestion from mock kafka consumer. - private void insertMockFiles(long fileTimestampA, long fileTimestampB) { - String path = config.getHdfsPath() + "/" + "testConsumerTopic"; // "hdfs:///opt/teragrep/cfe_39/srv/testConsumerTopic" - //Get the filesystem - HDFS - assertDoesNotThrow(() -> { - - // Sets the directory where the data should be stored, if the directory doesn't exist then it's created. - Path newDirectoryPath = new Path(path); - if (!fs.exists(newDirectoryPath)) { - // Create new Directory - fs.mkdirs(newDirectoryPath); - LOGGER.debug("Path {} created.", path); - } - - String dir = System.getProperty("user.dir") + "/src/test/java/com/teragrep/cfe_39/mockHdfsFiles"; - Set listOfFiles = Stream - .of(Objects.requireNonNull(new File(dir).listFiles())) - .filter(file -> !file.isDirectory()) - .map(File::getName) - .collect(Collectors.toSet()); - // Loop through all the avro files - for (String fileName : listOfFiles) { - String pathname = dir + "/" + fileName; - File avroFile = new File(pathname); - //==== Write file - LOGGER.debug("Begin Write file into hdfs"); - //Create a path - Path hdfswritepath = new Path(newDirectoryPath + "/" + avroFile.getName()); // filename should be set according to the requirements: 0.12345 where 0 is Kafka partition and 12345 is Kafka offset. - Assertions.assertFalse(fs.exists(hdfswritepath)); - Path readPath = new Path(avroFile.getPath()); - fs.copyFromLocalFile(readPath, hdfswritepath); - // Set fileTimestampA/fileTimestampB to something like 157784760000 to trigger pruning, -1 to not alter the timestamp. - if (Objects.equals(hdfswritepath.toString(), "hdfs:/opt/teragrep/cfe_39/srv/testConsumerTopic/0.9")) { - fs.setTimes(hdfswritepath, fileTimestampA, -1); - } - else if ( - Objects.equals(hdfswritepath.toString(), "hdfs:/opt/teragrep/cfe_39/srv/testConsumerTopic/0.13") - ) { - fs.setTimes(hdfswritepath, fileTimestampB, -1); - } - LOGGER.debug("End Write file into hdfs"); - LOGGER.debug("\nFile committed to HDFS, file writepath should be: {}\n", hdfswritepath); - } - }); - } } From 8b9eda33b5446dfbcbff690bb0083c1f6a761b6a Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Thu, 20 Jun 2024 12:13:24 +0300 Subject: [PATCH 100/146] Added codeblock --- README.adoc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.adoc b/README.adoc index 05db87ac..53bbf85b 100644 --- a/README.adoc +++ b/README.adoc @@ -14,7 +14,7 @@ See the official documentation on https://docs.teragrep.com[docs.teragrep.com]. ## How to [compile/use/implement] // TODO: add instructions how people can start to use your project, also add more information on the required configuration files. -mvn clean install +`mvn clean install` config.jaas, application.properties and log4j2.properties files have to created to use this module. The files must be placed in the working directory. From f57c4d7d80c4f1e3214d8a11ba8df244037965c8 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Thu, 20 Jun 2024 12:14:13 +0300 Subject: [PATCH 101/146] Added example config.jaas, application.properties and log4j2.properties files to rpm/resources --- rpm/resources/application.properties | 35 ++++++++++++++++++++++++++++ rpm/resources/config.jaas | 9 +++++++ rpm/resources/log4j2.properties | 10 ++++++++ 3 files changed, 54 insertions(+) create mode 100644 rpm/resources/application.properties create mode 100644 rpm/resources/config.jaas create mode 100644 rpm/resources/log4j2.properties diff --git a/rpm/resources/application.properties b/rpm/resources/application.properties new file mode 100644 index 00000000..c67167fa --- /dev/null +++ b/rpm/resources/application.properties @@ -0,0 +1,35 @@ +# Kafka security configuration file +java.security.auth.login.config=/home/p000048u/IdeaProjects/cfe_39/etc/config.jaas +# Logger settings +log4j2.configurationFile=/home/p000048u/IdeaProjects/cfe_39/etc/log4j2.properties +# What topics are searched from kafka, regex +queueTopicPattern=^testConsumerTopic-*$ +# Number of consumers created to the consumer groups +numOfConsumers=2 +# Kafka bootstrap servers - 127.0.0.1:9094,127.0.0.2:9094,127.0.0.3:9094 +consumer.bootstrap.servers=test +# Offset, should not be touched +consumer.auto.offset.reset=earliest +# Autocommit, should not be touched +consumer.enable.auto.commit=false +# Consumer group id, this is to track the progress of reading hte topic +consumer.group.id=cfe_39 +# Used security protocol and mechanism +consumer.security.protocol=SASL_PLAINTEXT +consumer.sasl.mechanism=PLAIN +# Maximum records per batch, note that too big number will cause massive load and can cause timeouts to trigger +consumer.max.poll.records=500 +# How much data can be fetched in one go +consumer.fetch.max.bytes=1073741820 +# How long for request before timing out. Note that too big max poll records size can cause this to trigger +consumer.request.timeout.ms=300000 +consumer.max.poll.interval.ms=300000 +# For testing only, remove for prod. +consumer.useMockKafkaConsumer=true +# AVRO +queueDirectory=/home/p000048u/IdeaProjects/cfe_39/etc/AVRO/ +queueNamePrefix=testingAVRO +# The maximum file size for AVRO-files that are to be stored in HDFS database. +maximumFileSize=3000 +# HDFS pruning, use 157784760000 value while testing HDFS writes to ensure the test records are not pruned. 157784760000L +pruneOffset=157784760000 \ No newline at end of file diff --git a/rpm/resources/config.jaas b/rpm/resources/config.jaas new file mode 100644 index 00000000..045b8540 --- /dev/null +++ b/rpm/resources/config.jaas @@ -0,0 +1,9 @@ +KafkaServer { + org.apache.kafka.common.security.plain.PlainLoginModule required + username="admin" + password="admin" + user_admin="admin" + user_alice="alice" + user_bob="bob" + user_charlie="charlie"; +}; \ No newline at end of file diff --git a/rpm/resources/log4j2.properties b/rpm/resources/log4j2.properties new file mode 100644 index 00000000..9ec3d8ec --- /dev/null +++ b/rpm/resources/log4j2.properties @@ -0,0 +1,10 @@ +appender.console.type = Console +appender.console.name = ConsoleLogger +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = %d{HH:mm:ss.SSS} [%t] %-5level %logger{36} - %msg%n +logging.level.org.apache.kafka=WARN +logging.level.io.confluent.kafka=WARN +rootLogger.level = INFO +rootLogger.appenderRef.stdout.ref = ConsoleLogger +logger.kafka.name = org.apache.kafka +logger.kafka.level = warn \ No newline at end of file From 54cc6b9ca71c49154bdb744f781e0760cded7ae9 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Thu, 20 Jun 2024 12:37:17 +0300 Subject: [PATCH 102/146] Added public visibility to Config() in Config.java. --- src/main/java/com/teragrep/cfe_39/Config.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/com/teragrep/cfe_39/Config.java b/src/main/java/com/teragrep/cfe_39/Config.java index 2c079feb..ff6e3423 100644 --- a/src/main/java/com/teragrep/cfe_39/Config.java +++ b/src/main/java/com/teragrep/cfe_39/Config.java @@ -81,7 +81,7 @@ public class Config { // TODO: Set up configuration check for important parameters. - Config() throws IOException { + public Config() throws IOException { Properties properties = new Properties(); Path configPath = Paths .get(System.getProperty("cfe_30.config.location", System.getProperty("user.dir") + "/etc/application.properties")); From 230960e193673e461ce04322c096d51f50a85e82 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Thu, 20 Jun 2024 13:09:04 +0300 Subject: [PATCH 103/146] Fixed typo in configPath parameter initialization. --- src/main/java/com/teragrep/cfe_39/Config.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/com/teragrep/cfe_39/Config.java b/src/main/java/com/teragrep/cfe_39/Config.java index ff6e3423..89f94400 100644 --- a/src/main/java/com/teragrep/cfe_39/Config.java +++ b/src/main/java/com/teragrep/cfe_39/Config.java @@ -84,7 +84,7 @@ public class Config { public Config() throws IOException { Properties properties = new Properties(); Path configPath = Paths - .get(System.getProperty("cfe_30.config.location", System.getProperty("user.dir") + "/etc/application.properties")); + .get(System.getProperty("cfe_39.config.location", System.getProperty("user.dir") + "/etc/application.properties")); LOGGER.info("Loading application config <[{}]>", configPath.toAbsolutePath()); try (InputStream inputStream = Files.newInputStream(configPath)) { From 7e981f1a60a58a0b3b5df471efb302ecf09c8a54 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Thu, 20 Jun 2024 14:15:50 +0300 Subject: [PATCH 104/146] Removed queueNamePrefix from config parameters, instead sourcing the queueNamePrefix value from topic name. Added fallback path to queueDirectory config parameter. --- rpm/resources/application.properties | 9 ++++----- src/main/java/com/teragrep/cfe_39/Config.java | 8 +------- .../teragrep/cfe_39/consumers/kafka/DatabaseOutput.java | 2 +- .../cfe_39/consumers/kafka/queue/WritableQueue.java | 4 ++-- 4 files changed, 8 insertions(+), 15 deletions(-) diff --git a/rpm/resources/application.properties b/rpm/resources/application.properties index c67167fa..e1f34301 100644 --- a/rpm/resources/application.properties +++ b/rpm/resources/application.properties @@ -1,12 +1,12 @@ # Kafka security configuration file -java.security.auth.login.config=/home/p000048u/IdeaProjects/cfe_39/etc/config.jaas +java.security.auth.login.config=/home/user/IdeaProjects/cfe_39/etc/config.jaas # Logger settings -log4j2.configurationFile=/home/p000048u/IdeaProjects/cfe_39/etc/log4j2.properties +log4j2.configurationFile=/home/user/IdeaProjects/cfe_39/etc/log4j2.properties # What topics are searched from kafka, regex queueTopicPattern=^testConsumerTopic-*$ # Number of consumers created to the consumer groups numOfConsumers=2 -# Kafka bootstrap servers - 127.0.0.1:9094,127.0.0.2:9094,127.0.0.3:9094 +# Kafka bootstrap servers consumer.bootstrap.servers=test # Offset, should not be touched consumer.auto.offset.reset=earliest @@ -27,8 +27,7 @@ consumer.max.poll.interval.ms=300000 # For testing only, remove for prod. consumer.useMockKafkaConsumer=true # AVRO -queueDirectory=/home/p000048u/IdeaProjects/cfe_39/etc/AVRO/ -queueNamePrefix=testingAVRO +queueDirectory=/home/user/IdeaProjects/cfe_39/etc/AVRO/ # The maximum file size for AVRO-files that are to be stored in HDFS database. maximumFileSize=3000 # HDFS pruning, use 157784760000 value while testing HDFS writes to ensure the test records are not pruned. 157784760000L diff --git a/src/main/java/com/teragrep/cfe_39/Config.java b/src/main/java/com/teragrep/cfe_39/Config.java index 89f94400..249828ac 100644 --- a/src/main/java/com/teragrep/cfe_39/Config.java +++ b/src/main/java/com/teragrep/cfe_39/Config.java @@ -66,7 +66,6 @@ public class Config { private final String hdfsPath; private String hdfsuri; private final String queueDirectory; - private final String queueNamePrefix; private final String kerberosHost; private final String kerberosRealm; private final String kerberosPrincipal; @@ -100,8 +99,7 @@ public Config() throws IOException { this.pruneOffset = Long.parseLong(properties.getProperty("pruneOffset", "172800000")); // AVRO - this.queueDirectory = properties.getProperty("queueDirectory", ""); - this.queueNamePrefix = properties.getProperty("queueNamePrefix", ""); + this.queueDirectory = properties.getProperty("queueDirectory", System.getProperty("user.dir") + "/etc/AVRO/"); this.maximumFileSize = Long.parseLong(properties.getProperty("maximumFileSize", "60800000")); // kerberos @@ -166,10 +164,6 @@ public String getQueueDirectory() { return queueDirectory; } - public String getQueueNamePrefix() { - return queueNamePrefix; - } - public String getQueueTopicPattern() { return queueTopicPattern; } diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java index 7fec9c70..136a236e 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java @@ -109,7 +109,7 @@ public DatabaseOutput( this.maximumFileSize = config.getMaximumFileSize(); // queueDirectory and queueNamePrefix are only used for temporarily storing the AVRO-serialized files before committing them to HDFS when the file size reaches the threshold (or all records are processed). - this.writableQueue = new WritableQueue(config.getQueueDirectory()); + this.writableQueue = new WritableQueue(config.getQueueDirectory(), table); this.sourceConcatenationBuffer = ByteBuffer.allocateDirect(256 * 1024); teragrepStreamName = new SDVector("teragrep@48577", "streamname"); diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/queue/WritableQueue.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/queue/WritableQueue.java index 6b2de85a..ecd32118 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/queue/WritableQueue.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/queue/WritableQueue.java @@ -67,9 +67,9 @@ public class WritableQueue { private final Path queueDirectory; private String queueNamePrefix; - public WritableQueue(String queueDirectory) { + public WritableQueue(String queueDirectory, String queueNamePrefix) { this.queueDirectory = Paths.get(queueDirectory); - this.queueNamePrefix = ""; + this.queueNamePrefix = queueNamePrefix; if (!Files.isDirectory(this.queueDirectory)) { throw new IllegalArgumentException("Provided path is not a " + "directory <[" + queueDirectory + "]>"); } From b0cc685fffde99946d500595902addc4f7ab6147 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Thu, 20 Jun 2024 14:38:27 +0300 Subject: [PATCH 105/146] Removed duplicate example configuration files from test section. --- .../teragrep/cfe_39/application.properties | 35 ------------------- src/test/java/com/teragrep/cfe_39/config.jaas | 9 ----- .../com/teragrep/cfe_39/log4j2.properties | 10 ------ 3 files changed, 54 deletions(-) delete mode 100644 src/test/java/com/teragrep/cfe_39/application.properties delete mode 100644 src/test/java/com/teragrep/cfe_39/config.jaas delete mode 100644 src/test/java/com/teragrep/cfe_39/log4j2.properties diff --git a/src/test/java/com/teragrep/cfe_39/application.properties b/src/test/java/com/teragrep/cfe_39/application.properties deleted file mode 100644 index 4cc21a53..00000000 --- a/src/test/java/com/teragrep/cfe_39/application.properties +++ /dev/null @@ -1,35 +0,0 @@ -# Kafka security configuration file -java.security.auth.login.config=/example_path/cfe_39/etc/config.jaas -# Logger settings -log4j2.configurationFile=/example_path/cfe_39/etc/log4j2.properties -# What topics are searched from kafka, regex -queueTopicPattern=^testConsumerTopic-*$ -# Number of consumers created to the consumer groups -numOfConsumers=2 -# Kafka bootstrap servers - 127.0.0.1:9094,127.0.0.2:9094,127.0.0.3:9094 -consumer.bootstrap.servers=test -# Offset, should not be touched -consumer.auto.offset.reset=earliest -# Autocommit, should not be touched -consumer.enable.auto.commit=false -# Consumer group id, this is to track the progress of reading hte topic -consumer.group.id=cfe_39 -# Used security protocol and mechanism -consumer.security.protocol=SASL_PLAINTEXT -consumer.sasl.mechanism=PLAIN -# Maximum records per batch, note that too big number will cause massive load and can cause timeouts to trigger -consumer.max.poll.records=500 -# How much data can be fetched in one go -consumer.fetch.max.bytes=1073741820 -# How long for request before timing out. Note that too big max poll records size can cause this to trigger -consumer.request.timeout.ms=300000 -consumer.max.poll.interval.ms=300000 -# For testing only, remove for prod. -consumer.useMockKafkaConsumer=true -# AVRO -queueDirectory=/example_path/cfe_39/etc/AVRO/ -queueNamePrefix=testingAVRO -# The maximum file size for AVRO-files that are to be stored in HDFS database. -maximumFileSize=3000 -# HDFS pruning, use 157784760000 value while testing HDFS writes to ensure the test records are not pruned. -pruneOffset=157784760000 \ No newline at end of file diff --git a/src/test/java/com/teragrep/cfe_39/config.jaas b/src/test/java/com/teragrep/cfe_39/config.jaas deleted file mode 100644 index 045b8540..00000000 --- a/src/test/java/com/teragrep/cfe_39/config.jaas +++ /dev/null @@ -1,9 +0,0 @@ -KafkaServer { - org.apache.kafka.common.security.plain.PlainLoginModule required - username="admin" - password="admin" - user_admin="admin" - user_alice="alice" - user_bob="bob" - user_charlie="charlie"; -}; \ No newline at end of file diff --git a/src/test/java/com/teragrep/cfe_39/log4j2.properties b/src/test/java/com/teragrep/cfe_39/log4j2.properties deleted file mode 100644 index 9ec3d8ec..00000000 --- a/src/test/java/com/teragrep/cfe_39/log4j2.properties +++ /dev/null @@ -1,10 +0,0 @@ -appender.console.type = Console -appender.console.name = ConsoleLogger -appender.console.layout.type = PatternLayout -appender.console.layout.pattern = %d{HH:mm:ss.SSS} [%t] %-5level %logger{36} - %msg%n -logging.level.org.apache.kafka=WARN -logging.level.io.confluent.kafka=WARN -rootLogger.level = INFO -rootLogger.appenderRef.stdout.ref = ConsoleLogger -logger.kafka.name = org.apache.kafka -logger.kafka.level = warn \ No newline at end of file From fdc919f2c7731ecc99ac7177d5d3ed6785984468 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Thu, 20 Jun 2024 15:09:49 +0300 Subject: [PATCH 106/146] Set visibility to private on committedToHdfs() in DatabaseOutput.java. --- .../com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java index 136a236e..37ea3951 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java @@ -124,7 +124,7 @@ public DatabaseOutput( } // Checks that the filesize stays under the defined maximum file size. If the file is about to go over target limit commits the file to HDFS and returns true, otherwise does nothing and returns false. - boolean committedToHdfs(long fileSize, JsonObject recordOffsetObjectJo) { + private boolean committedToHdfs(long fileSize, JsonObject recordOffsetObjectJo) { try { // If the syslogAvroWriter is already initialized, check the filesize so it doesn't go above maximumFileSize. if (fileSize > maximumFileSize) { From 03e4c202c95f452b24a0c29d507dd7b6ff431926 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Mon, 24 Jun 2024 12:16:47 +0300 Subject: [PATCH 107/146] Renamed committedToHdfs to writeToHdfs. Added missing second & to a condition. Moved generation of a new syslogAvroWriter outside the writeToHdfs. --- .../consumers/kafka/DatabaseOutput.java | 24 +++++++++++-------- 1 file changed, 14 insertions(+), 10 deletions(-) diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java index 37ea3951..197b5711 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java @@ -124,7 +124,7 @@ public DatabaseOutput( } // Checks that the filesize stays under the defined maximum file size. If the file is about to go over target limit commits the file to HDFS and returns true, otherwise does nothing and returns false. - private boolean committedToHdfs(long fileSize, JsonObject recordOffsetObjectJo) { + private boolean writeToHdfs(long fileSize, JsonObject recordOffsetObjectJo) { try { // If the syslogAvroWriter is already initialized, check the filesize so it doesn't go above maximumFileSize. if (fileSize > maximumFileSize) { @@ -135,12 +135,6 @@ private boolean committedToHdfs(long fileSize, JsonObject recordOffsetObjectJo) try (HDFSWrite writer = new HDFSWrite(config, recordOffsetObjectJo)) { writer.commit(syslogFile); // commits the final AVRO-file to HDFS. } - - // This part defines a new empty file to which the new AVRO-serialized records are stored until it again hits the 64M size limit. - writableQueue - .setQueueNamePrefix(recordOffsetObjectJo.get("topic").getAsString() + recordOffsetObjectJo.get("partition").getAsString()); - syslogFile = writableQueue.getNextWritableFile(); - syslogAvroWriter = new SyslogAvroWriter(syslogFile); return true; } } @@ -212,11 +206,16 @@ public void accept(List recordOffsetObjectList) { try { if ( lastObjectJo.get("topic").getAsString().equals(recordOffsetObjectJo.get("topic").getAsString()) - & lastObjectJo.get("partition").getAsString().equals(recordOffsetObjectJo.get("partition").getAsString()) + && lastObjectJo.get("partition").getAsString().equals(recordOffsetObjectJo.get("partition").getAsString()) ) { // Records left to consume in the current partition. - boolean fileCommitted = committedToHdfs(syslogAvroWriter.getFileSize(), lastObjectJo); + boolean fileCommitted = writeToHdfs(syslogAvroWriter.getFileSize(), lastObjectJo); if (fileCommitted) { + // This part defines a new empty file to which the new AVRO-serialized records are stored until it again hits the size limit defined in config. + writableQueue + .setQueueNamePrefix(recordOffsetObjectJo.get("topic").getAsString() + recordOffsetObjectJo.get("partition").getAsString()); + syslogFile = writableQueue.getNextWritableFile(); + syslogAvroWriter = new SyslogAvroWriter(syslogFile); if (LOGGER.isDebugEnabled()) { LOGGER .debug( @@ -291,8 +290,13 @@ public void accept(List recordOffsetObjectList) { // Calculate the size of syslogRecord that is going to be written to syslogAvroWriter-file. long capacity = syslogRecord.toByteBuffer().capacity(); // Check if there is still room in syslogAvroWriter for another syslogRecord. Commit syslogAvroWriter to HDFS if no room left, emptying it out in the process. - boolean fileCommitted = committedToHdfs(syslogAvroWriter.getFileSize() + capacity, lastObjectJo); + boolean fileCommitted = writeToHdfs(syslogAvroWriter.getFileSize() + capacity, lastObjectJo); if (fileCommitted) { + // This part defines a new empty file to which the new AVRO-serialized records are stored until it again hits the size limit defined in config. + writableQueue + .setQueueNamePrefix(recordOffsetObjectJo.get("topic").getAsString() + recordOffsetObjectJo.get("partition").getAsString()); + syslogFile = writableQueue.getNextWritableFile(); + syslogAvroWriter = new SyslogAvroWriter(syslogFile); if (LOGGER.isDebugEnabled()) { LOGGER .debug( From 3385811c50ab254212d14a973c217190b6ac7a64 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Mon, 24 Jun 2024 12:18:25 +0300 Subject: [PATCH 108/146] Added additional ingestion test for low maximum file size. --- .../teragrep/cfe_39/Ingestion0FilesTest.java | 63 +++++++++++++++++++ 1 file changed, 63 insertions(+) diff --git a/src/test/java/com/teragrep/cfe_39/Ingestion0FilesTest.java b/src/test/java/com/teragrep/cfe_39/Ingestion0FilesTest.java index be528bd0..21159b72 100644 --- a/src/test/java/com/teragrep/cfe_39/Ingestion0FilesTest.java +++ b/src/test/java/com/teragrep/cfe_39/Ingestion0FilesTest.java @@ -326,4 +326,67 @@ record = reader.next(record); Assertions.assertEquals(10, partitionCounter); }); } + + @Test + public void ingestion0FilesLowSizeTest() { + // Empty HDFS database, 140 records in mock kafka consumer ready for ingestion. All 14 records for each 10 topic partitions are stored in two avro-files per partition based on MaximumFileSize. + assertDoesNotThrow(() -> { + Assertions.assertTrue(config.getPruneOffset() >= 300000L); // Fails the test if the config is not correct. + config.setMaximumFileSize(3000); // This parameter defines the amount of records that can fit inside a single AVRO-file. + Assertions.assertFalse(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic"))); + HdfsDataIngestion hdfsDataIngestion = new HdfsDataIngestion(config); + Thread.sleep(10000); + hdfsDataIngestion.run(); + }); + + // Assert that the kafka records were ingested correctly and the database holds the correct 140 records. + + // Check that the files were properly written to HDFS. + String hdfsuri = config.getHdfsuri(); + + String path = config.getHdfsPath() + "/" + "testConsumerTopic"; + // ====== Init HDFS File System Object + Configuration conf = new Configuration(); + // Set FileSystem URI + conf.set("fs.defaultFS", hdfsuri); + // Because of Maven + conf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); + conf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); + // Set HADOOP user + System.setProperty("HADOOP_USER_NAME", "hdfs"); + System.setProperty("hadoop.home.dir", "/"); + //Get the filesystem - HDFS + assertDoesNotThrow(() -> { + fs = FileSystem.get(URI.create(hdfsuri), conf); + + Path workingDir = fs.getWorkingDirectory(); + Path newDirectoryPath = new Path(path); + Assertions.assertTrue(fs.exists(newDirectoryPath)); + + // Assert that the kafka records were ingested correctly and the database holds the expected 20 files. + Assertions + .assertEquals(20, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.13"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "1.9"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "1.13"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "2.9"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "2.13"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "3.9"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "3.13"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "4.9"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "4.13"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "5.9"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "5.13"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "6.9"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "6.13"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "7.9"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "7.13"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "8.9"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "8.13"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "9.9"))); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "9.13"))); + LOGGER.debug("All expected files present in HDFS."); + }); + } } From 678c2ae2d23101780eb04ce6a0686f733b3c63ea Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Tue, 25 Jun 2024 09:36:48 +0300 Subject: [PATCH 109/146] Refactored HDFSPrune.java to use dependency injection for FileSystem and renamed a single letter parameter. --- .../cfe_39/consumers/kafka/HDFSPrune.java | 108 +++--------------- .../cfe_39/consumers/kafka/HDFSRead.java | 9 +- .../consumers/kafka/HdfsDataIngestion.java | 48 +++++++- .../teragrep/cfe_39/PruningNoFilesTest.java | 2 +- .../cfe_39/PruningOneNewFileTest.java | 2 +- .../cfe_39/PruningOneOldFileTest.java | 2 +- .../cfe_39/PruningOneOldOneNewFileTest.java | 2 +- .../cfe_39/PruningTwoNewFilesTest.java | 2 +- .../cfe_39/PruningTwoOldFilesTest.java | 2 +- 9 files changed, 70 insertions(+), 107 deletions(-) diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSPrune.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSPrune.java index 38c1988a..7a3fddb6 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSPrune.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSPrune.java @@ -46,105 +46,31 @@ package com.teragrep.cfe_39.consumers.kafka; import com.teragrep.cfe_39.Config; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.LocalFileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hdfs.DistributedFileSystem; -import org.apache.hadoop.security.UserGroupInformation; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; -import java.net.URI; -import java.util.Properties; public class HDFSPrune { private static final Logger LOGGER = LoggerFactory.getLogger(HDFSPrune.class); - private Config config; private final FileSystem fs; - private Path newDirectoryPath; - private long cutOffEpoch; - private final boolean useMockKafkaConsumer; // test-mode switch - - public HDFSPrune(Config config, String topicName) throws IOException { - - // Check for testmode from config. - Properties readerKafkaProperties = config.getKafkaConsumerProperties(); - this.useMockKafkaConsumer = Boolean - .parseBoolean(readerKafkaProperties.getProperty("useMockKafkaConsumer", "false")); - - if (useMockKafkaConsumer) { - this.config = config; - String hdfsuri = config.getHdfsuri(); - String path = config.getHdfsPath().concat("/").concat(topicName); - // ====== Init HDFS File System Object - Configuration conf = new Configuration(); - // Set FileSystem URI - conf.set("fs.defaultFS", hdfsuri); - // Because of Maven - conf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); - conf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); - // Set HADOOP user - System.setProperty("HADOOP_USER_NAME", "hdfs"); - System.setProperty("hadoop.home.dir", "/"); - //Get the filesystem - HDFS - fs = FileSystem.get(URI.create(hdfsuri), conf); - - //==== Create directory if not exists - Path workingDir = fs.getWorkingDirectory(); - newDirectoryPath = new Path(path); - if (!fs.exists(newDirectoryPath)) { - // Create new Directory - fs.mkdirs(newDirectoryPath); - LOGGER.info("Path <{}> created.", path); - } - } - else { - // Code for initializing the class with kerberos. - String hdfsuri = config.getHdfsuri(); // Get from config. - - String path = config.getHdfsPath() + "/" + topicName; - - // set kerberos host and realm - System.setProperty("java.security.krb5.realm", config.getKerberosRealm()); - System.setProperty("java.security.krb5.kdc", config.getKerberosHost()); - - Configuration conf = new Configuration(); - - // enable kerberus - conf.set("hadoop.security.authentication", config.getHadoopAuthentication()); - conf.set("hadoop.security.authorization", config.getHadoopAuthorization()); - - conf.set("fs.defaultFS", hdfsuri); // Set FileSystem URI - conf.set("fs.hdfs.impl", DistributedFileSystem.class.getName()); // Maven stuff? - conf.set("fs.file.impl", LocalFileSystem.class.getName()); // Maven stuff? - - /* hack for running locally with fake DNS records - set this to true if overriding the host name in /etc/hosts*/ - conf.set("dfs.client.use.datanode.hostname", config.getKerberosTestMode()); - - /* server principal - the kerberos principle that the namenode is using*/ - conf.set("dfs.namenode.kerberos.principal.pattern", config.getKerberosPrincipal()); - - // set usergroup stuff - UserGroupInformation.setConfiguration(conf); - UserGroupInformation.loginUserFromKeytab(config.getKerberosKeytabUser(), config.getKerberosKeytabPath()); - - // filesystem for HDFS access is set here - fs = FileSystem.get(conf); - - //==== Create directory if not exists - Path workingDir = fs.getWorkingDirectory(); - newDirectoryPath = new Path(path); - if (!fs.exists(newDirectoryPath)) { - // Create new Directory - fs.mkdirs(newDirectoryPath); - LOGGER.info("Path <{}> created.", path); - } + private final Path newDirectoryPath; + private final long cutOffEpoch; + + public HDFSPrune(Config config, String topicName, FileSystem fs) throws IOException { + this.fs = fs; + String path = config.getHdfsPath().concat("/").concat(topicName); + //==== Create directory if not exists + Path workingDir = fs.getWorkingDirectory(); + newDirectoryPath = new Path(path); + if (!fs.exists(newDirectoryPath)) { + // Create new Directory + fs.mkdirs(newDirectoryPath); + LOGGER.info("Path <{}> created.", path); } long pruneOffset = config.getPruneOffset(); cutOffEpoch = System.currentTimeMillis() - pruneOffset; // pruneOffset is parametrized in Config.java. Default value is 2 days in milliseconds. @@ -155,11 +81,11 @@ public int prune() throws IOException { // Fetch the filestatuses of HDFS files. FileStatus[] fileStatuses = fs.listStatus(new Path(newDirectoryPath + "/")); if (fileStatuses.length > 0) { - for (FileStatus a : fileStatuses) { + for (FileStatus fileStatus : fileStatuses) { // Delete old files - if (a.getModificationTime() < cutOffEpoch) { - boolean delete = fs.delete(a.getPath(), true); - LOGGER.info("Deleted file <{}>", a.getPath()); + if (fileStatus.getModificationTime() < cutOffEpoch) { + boolean delete = fs.delete(fileStatus.getPath(), true); + LOGGER.info("Deleted file <{}>", fileStatus.getPath()); deleted++; } } diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSRead.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSRead.java index 8517c106..0ab7ec77 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSRead.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSRead.java @@ -193,13 +193,6 @@ public boolean accept(Path path) { // try-with-resources handles closing the filesystem automatically. public void close() { - if (fs != null) { - try { - fs.close(); - } - catch (IOException e) { - throw new RuntimeException(e); - } - } + // NoOp, as closing the FileSystem object here would also close all the other FileSystem objects. } } diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HdfsDataIngestion.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HdfsDataIngestion.java index e2c1d2fe..0f5190ea 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HdfsDataIngestion.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HdfsDataIngestion.java @@ -48,6 +48,11 @@ import com.teragrep.cfe_39.Config; import com.teragrep.cfe_39.metrics.*; import com.teragrep.cfe_39.metrics.topic.TopicCounter; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocalFileSystem; +import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.apache.hadoop.security.UserGroupInformation; import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; @@ -56,6 +61,7 @@ import org.slf4j.LoggerFactory; import java.io.IOException; +import java.net.URI; import java.sql.SQLException; import java.time.Duration; import java.util.*; @@ -76,8 +82,9 @@ public class HdfsDataIngestion { private boolean useMockKafkaConsumer; private final int numOfConsumers; private Map hdfsStartOffsets; + private final FileSystem fs; - public HdfsDataIngestion(Config config) { + public HdfsDataIngestion(Config config) throws IOException { keepRunning = true; this.config = config; Properties readerKafkaProperties = config.getKafkaConsumerProperties(); @@ -86,6 +93,20 @@ public HdfsDataIngestion(Config config) { .parseBoolean(readerKafkaProperties.getProperty("useMockKafkaConsumer", "false")); if (useMockKafkaConsumer) { this.kafkaConsumer = MockKafkaConsumerFactory.getConsumer(0); // A consumer used only for scanning the available topics to be allocated to consumers running in different threads (thus 0 as input parameter). + // Initializing the FileSystem with minicluster. + String hdfsuri = config.getHdfsuri(); + // ====== Init HDFS File System Object + Configuration conf = new Configuration(); + // Set FileSystem URI + conf.set("fs.defaultFS", hdfsuri); + // Because of Maven + conf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); + conf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); + // Set HADOOP user + System.setProperty("HADOOP_USER_NAME", "hdfs"); + System.setProperty("hadoop.home.dir", "/"); + //Get the filesystem - HDFS + fs = FileSystem.get(URI.create(hdfsuri), conf); } else { this.kafkaConsumer = new KafkaConsumer<>( @@ -93,6 +114,29 @@ public HdfsDataIngestion(Config config) { new ByteArrayDeserializer(), new ByteArrayDeserializer() ); + // Initializing the FileSystem with kerberos. + String hdfsuri = config.getHdfsuri(); // Get from config. + // set kerberos host and realm + System.setProperty("java.security.krb5.realm", config.getKerberosRealm()); + System.setProperty("java.security.krb5.kdc", config.getKerberosHost()); + Configuration conf = new Configuration(); + // enable kerberus + conf.set("hadoop.security.authentication", config.getHadoopAuthentication()); + conf.set("hadoop.security.authorization", config.getHadoopAuthorization()); + conf.set("fs.defaultFS", hdfsuri); // Set FileSystem URI + conf.set("fs.hdfs.impl", DistributedFileSystem.class.getName()); // Maven stuff? + conf.set("fs.file.impl", LocalFileSystem.class.getName()); // Maven stuff? + /* hack for running locally with fake DNS records + set this to true if overriding the host name in /etc/hosts*/ + conf.set("dfs.client.use.datanode.hostname", config.getKerberosTestMode()); + /* server principal + the kerberos principle that the namenode is using*/ + conf.set("dfs.namenode.kerberos.principal.pattern", config.getKerberosPrincipal()); + // set usergroup stuff + UserGroupInformation.setConfiguration(conf); + UserGroupInformation.loginUserFromKeytab(config.getKerberosKeytabUser(), config.getKerberosKeytabPath()); + // filesystem for HDFS access is set here + fs = FileSystem.get(conf); } hdfsStartOffsets = new HashMap<>(); } @@ -127,7 +171,7 @@ public void run() throws InterruptedException { LOGGER.info("topic that is being pruned: <{}>", topic_name); if (topic_name != null) { try { - HDFSPrune hdfsPrune = new HDFSPrune(config, topic_name); + HDFSPrune hdfsPrune = new HDFSPrune(config, topic_name, fs); hdfsPrune.prune(); } catch (IOException e) { diff --git a/src/test/java/com/teragrep/cfe_39/PruningNoFilesTest.java b/src/test/java/com/teragrep/cfe_39/PruningNoFilesTest.java index 671e84ea..a3472544 100644 --- a/src/test/java/com/teragrep/cfe_39/PruningNoFilesTest.java +++ b/src/test/java/com/teragrep/cfe_39/PruningNoFilesTest.java @@ -115,7 +115,7 @@ public void teardownMiniCluster() { public void noFiles() { assertDoesNotThrow(() -> { Assertions.assertFalse(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic"))); - HDFSPrune hdfsPrune = new HDFSPrune(config, "testConsumerTopic"); + HDFSPrune hdfsPrune = new HDFSPrune(config, "testConsumerTopic", fs); Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic"))); Assertions .assertEquals(0, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); diff --git a/src/test/java/com/teragrep/cfe_39/PruningOneNewFileTest.java b/src/test/java/com/teragrep/cfe_39/PruningOneNewFileTest.java index a4e40abb..8b106c8a 100644 --- a/src/test/java/com/teragrep/cfe_39/PruningOneNewFileTest.java +++ b/src/test/java/com/teragrep/cfe_39/PruningOneNewFileTest.java @@ -153,7 +153,7 @@ public void oneNewFileTest() { Assertions .assertEquals(1, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); - HDFSPrune hdfsPrune = new HDFSPrune(config, "testConsumerTopic"); + HDFSPrune hdfsPrune = new HDFSPrune(config, "testConsumerTopic", fs); int deleted = hdfsPrune.prune(); Assertions.assertEquals(0, deleted); // Also check with HDFS access if expected files still exist. diff --git a/src/test/java/com/teragrep/cfe_39/PruningOneOldFileTest.java b/src/test/java/com/teragrep/cfe_39/PruningOneOldFileTest.java index 516523c8..eefc42da 100644 --- a/src/test/java/com/teragrep/cfe_39/PruningOneOldFileTest.java +++ b/src/test/java/com/teragrep/cfe_39/PruningOneOldFileTest.java @@ -153,7 +153,7 @@ public void oneOldFileTest() { Assertions .assertEquals(1, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); - HDFSPrune hdfsPrune = new HDFSPrune(config, "testConsumerTopic"); + HDFSPrune hdfsPrune = new HDFSPrune(config, "testConsumerTopic", fs); int deleted = hdfsPrune.prune(); Assertions.assertEquals(1, deleted); // Also check with HDFS access if expected files still exist. diff --git a/src/test/java/com/teragrep/cfe_39/PruningOneOldOneNewFileTest.java b/src/test/java/com/teragrep/cfe_39/PruningOneOldOneNewFileTest.java index 1fca4dd4..e2f9c69f 100644 --- a/src/test/java/com/teragrep/cfe_39/PruningOneOldOneNewFileTest.java +++ b/src/test/java/com/teragrep/cfe_39/PruningOneOldOneNewFileTest.java @@ -157,7 +157,7 @@ public void oneOldOneNewFileTest() { .assertEquals(2, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.13"))); - HDFSPrune hdfsPrune = new HDFSPrune(config, "testConsumerTopic"); + HDFSPrune hdfsPrune = new HDFSPrune(config, "testConsumerTopic", fs); int deleted = hdfsPrune.prune(); Assertions.assertEquals(1, deleted); // Also check with HDFS access if expected files still exist. diff --git a/src/test/java/com/teragrep/cfe_39/PruningTwoNewFilesTest.java b/src/test/java/com/teragrep/cfe_39/PruningTwoNewFilesTest.java index 06e47f92..0e042bb3 100644 --- a/src/test/java/com/teragrep/cfe_39/PruningTwoNewFilesTest.java +++ b/src/test/java/com/teragrep/cfe_39/PruningTwoNewFilesTest.java @@ -155,7 +155,7 @@ public void twoNewFilesTest() { .assertEquals(2, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.13"))); - HDFSPrune hdfsPrune = new HDFSPrune(config, "testConsumerTopic"); + HDFSPrune hdfsPrune = new HDFSPrune(config, "testConsumerTopic", fs); int deleted = hdfsPrune.prune(); Assertions.assertEquals(0, deleted); // Also check with HDFS access if expected files still exist. diff --git a/src/test/java/com/teragrep/cfe_39/PruningTwoOldFilesTest.java b/src/test/java/com/teragrep/cfe_39/PruningTwoOldFilesTest.java index eb4804ea..1f0b8624 100644 --- a/src/test/java/com/teragrep/cfe_39/PruningTwoOldFilesTest.java +++ b/src/test/java/com/teragrep/cfe_39/PruningTwoOldFilesTest.java @@ -156,7 +156,7 @@ public void twoOldFilesTest() { .assertEquals(2, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.13"))); - HDFSPrune hdfsPrune = new HDFSPrune(config, "testConsumerTopic"); + HDFSPrune hdfsPrune = new HDFSPrune(config, "testConsumerTopic", fs); int deleted = hdfsPrune.prune(); Assertions.assertEquals(2, deleted); // Also check with HDFS access if expected files still exist. From 5b9025d4b47b63a2588bfb597d82901fda23da58 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Tue, 25 Jun 2024 09:56:02 +0300 Subject: [PATCH 110/146] Refactored HDFSRead.java. Added dependency injection for FileSystem, removed unused regex filtering and renamed single letter variables. --- .../cfe_39/consumers/kafka/HDFSRead.java | 99 ++----------------- .../consumers/kafka/HdfsDataIngestion.java | 2 +- 2 files changed, 10 insertions(+), 91 deletions(-) diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSRead.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSRead.java index 0ab7ec77..bf1f92f6 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSRead.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSRead.java @@ -46,19 +46,14 @@ package com.teragrep.cfe_39.consumers.kafka; import com.teragrep.cfe_39.Config; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.*; -import org.apache.hadoop.hdfs.DistributedFileSystem; -import org.apache.hadoop.security.UserGroupInformation; import org.apache.kafka.common.TopicPartition; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; -import java.net.URI; import java.util.HashMap; import java.util.Map; -import java.util.Properties; public final class HDFSRead implements AutoCloseable { /* Maps out the latest offset for all the topic partitions available in HDFS. @@ -67,83 +62,15 @@ The offset map can then be used for kafka consumer seek() method, which will add private static final Logger LOGGER = LoggerFactory.getLogger(HDFSRead.class); private final FileSystem fs; - private final boolean useMockKafkaConsumer; // test-mode switch - private final Configuration conf; - private final String hdfsuri; - private static String topicsRegexString = null; private final String path; - public HDFSRead(Config config) throws IOException { - // Check if mock kafka consumer is enabled in the config. - Properties readerKafkaProperties = config.getKafkaConsumerProperties(); - this.useMockKafkaConsumer = Boolean - .parseBoolean(readerKafkaProperties.getProperty("useMockKafkaConsumer", "false")); - - if (useMockKafkaConsumer) { - // Code for initializing the class in test mode without kerberos. - hdfsuri = config.getHdfsuri(); // Get from config. - path = config.getHdfsPath(); - - // ====== Init HDFS File System Object - conf = new Configuration(); - // Set FileSystem URI - conf.set("fs.defaultFS", hdfsuri); - // Because of Maven - conf.set("fs.hdfs.impl", DistributedFileSystem.class.getName()); - conf.set("fs.file.impl", LocalFileSystem.class.getName()); - // Set HADOOP user here, Kerberus parameters most likely needs to be added here too. - System.setProperty("HADOOP_USER_NAME", "hdfs"); - System.setProperty("hadoop.home.dir", "/"); - // filesystem for HDFS access is set here - try { - fs = FileSystem.get(URI.create(hdfsuri), conf); - } - catch (IOException e) { - throw new RuntimeException(e); - } - - } - else { - // Code for initializing the class with kerberos. - hdfsuri = config.getHdfsuri(); // Get from config.' - path = config.getHdfsPath(); - - // set kerberos host and realm - System.setProperty("java.security.krb5.realm", config.getKerberosRealm()); - System.setProperty("java.security.krb5.kdc", config.getKerberosHost()); - - conf = new Configuration(); - - // enable kerberus - conf.set("hadoop.security.authentication", config.getHadoopAuthentication()); - conf.set("hadoop.security.authorization", config.getHadoopAuthorization()); - - conf.set("fs.defaultFS", hdfsuri); // Set FileSystem URI - conf.set("fs.hdfs.impl", DistributedFileSystem.class.getName()); // Maven stuff? - conf.set("fs.file.impl", LocalFileSystem.class.getName()); // Maven stuff? - - /* hack for running locally with fake DNS records - set this to true if overriding the host name in /etc/hosts*/ - conf.set("dfs.client.use.datanode.hostname", config.getKerberosTestMode()); - - // server principal - // the kerberos principle that the namenode is using - conf.set("dfs.namenode.kerberos.principal.pattern", config.getKerberosPrincipal()); - - // set usergroup stuff - UserGroupInformation.setConfiguration(conf); - UserGroupInformation.loginUserFromKeytab(config.getKerberosKeytabUser(), config.getKerberosKeytabPath()); - - // filesystem for HDFS access is set here - fs = FileSystem.get(conf); - } + public HDFSRead(Config config, FileSystem fs) throws IOException { + this.fs = fs; + path = config.getHdfsPath(); } public Map hdfsStartOffsets() throws IOException { Map offsets = new HashMap<>(); - if (topicsRegexString == null) { - topicsRegexString = "^.*$"; // FIXME: all topics if none given - } Path workingDir = fs.getWorkingDirectory(); Path newDirectoryPath = new Path(path); @@ -153,16 +80,16 @@ public Map hdfsStartOffsets() throws IOException { LOGGER.info("Path <{}> created.", path); } - FileStatus[] directoryStatuses = fs.listStatus(new Path(path), topicFilter); + FileStatus[] directoryStatuses = fs.listStatus(new Path(path)); // Get the directory statuses. Each directory represents a Kafka topic. if (directoryStatuses.length > 0) { LOGGER.debug("Found <{}> matching directories", directoryStatuses.length); - for (FileStatus r : directoryStatuses) { + for (FileStatus directoryStatus : directoryStatuses) { // Get the file statuses that are inside the directories. - FileStatus[] fileStatuses = fs.listStatus(r.getPath()); - for (FileStatus r2 : fileStatuses) { - String topic = r2.getPath().getParent().getName(); - String[] split = r2.getPath().getName().split("\\."); // The file name can be split to partition parameter and offset parameter. First value is partition and second is offset. + FileStatus[] fileStatuses = fs.listStatus(directoryStatus.getPath()); + for (FileStatus fileStatus : fileStatuses) { + String topic = fileStatus.getPath().getParent().getName(); + String[] split = fileStatus.getPath().getName().split("\\."); // The file name can be split to partition parameter and offset parameter. First value is partition and second is offset. String partition = split[0]; String offset = split[1]; TopicPartition topicPartition = new TopicPartition(topic, Integer.parseInt(partition)); @@ -183,14 +110,6 @@ public Map hdfsStartOffsets() throws IOException { return offsets; } - private static final PathFilter topicFilter = new PathFilter() { - - @Override - public boolean accept(Path path) { - return path.getName().matches(topicsRegexString); // Catches the directory names. - } - }; - // try-with-resources handles closing the filesystem automatically. public void close() { // NoOp, as closing the FileSystem object here would also close all the other FileSystem objects. diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HdfsDataIngestion.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HdfsDataIngestion.java index 0f5190ea..ada6dad6 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HdfsDataIngestion.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HdfsDataIngestion.java @@ -151,7 +151,7 @@ public void run() throws InterruptedException { List topicCounters = new CopyOnWriteArrayList<>(); // Generates offsets of the already committed records for Kafka and passes them to the kafka consumers. - try (HDFSRead hr = new HDFSRead(config)) { + try (HDFSRead hr = new HDFSRead(config, fs)) { hdfsStartOffsets = hr.hdfsStartOffsets(); LOGGER.debug("topicPartitionStartMap generated succesfully: <{}>", hdfsStartOffsets); } From 38de4b0d6ec616700acfc73a333246f1e17b924b Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Tue, 25 Jun 2024 12:19:09 +0300 Subject: [PATCH 111/146] Renamed thread id parameter from testi to threadId. --- .../teragrep/cfe_39/consumers/kafka/HdfsDataIngestion.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HdfsDataIngestion.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HdfsDataIngestion.java index ada6dad6..0c60578f 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HdfsDataIngestion.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HdfsDataIngestion.java @@ -204,7 +204,7 @@ private void createReader( /* Every consumer is run in a separate thread. Consumer group is also handled here, and each consumer of the group runs on separate thread.*/ int numOfThreads = Math.min(numOfConsumers, listPartitionInfo.size()); // Makes sure that there aren't more consumers than available partitions in the consumer group. - for (int testi = 1; numOfThreads >= testi; testi++) { + for (int threadId = 1; numOfThreads >= threadId; threadId++) { Consumer> output = new DatabaseOutput( config, // Configuration settings topic, // String, the name of the topic @@ -217,7 +217,7 @@ private void createReader( output, hdfsStartOffsets ); - Thread readThread = new Thread(null, readCoordinator, topic + testi); // Starts the thread with readCoordinator that creates the consumer and subscribes to the topic. + Thread readThread = new Thread(null, readCoordinator, topic + threadId); // Starts the thread with readCoordinator that creates the consumer and subscribes to the topic. threads.add(readThread); readThread.start(); // Starts the thread, in other words proceeds to call run() function of ReadCoordinator. } From d6b7e7ab9deb00f5d5cd62722b6095e24b7ef3e6 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Tue, 25 Jun 2024 14:12:45 +0300 Subject: [PATCH 112/146] Removed unused stop() method. --- .../com/teragrep/cfe_39/consumers/kafka/ReadCoordinator.java | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinator.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinator.java index e28ca98b..232c83bd 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinator.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/ReadCoordinator.java @@ -134,9 +134,4 @@ public void run() { } } } - - // FIXME: remove? - public void stop() { - run = false; - } } From ae2b625f9bd72d9b30e1ad2a50468b61e995eb17 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Tue, 25 Jun 2024 15:04:22 +0300 Subject: [PATCH 113/146] Changed RecordOffset json printer to use numeric values for partition and offset instead of string. --- .../cfe_39/consumers/kafka/RecordOffset.java | 5 +-- .../teragrep/cfe_39/KafkaConsumerTest.java | 40 +++++++++---------- 2 files changed, 21 insertions(+), 24 deletions(-) diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/RecordOffset.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/RecordOffset.java index 5f3369c0..543a58fd 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/RecordOffset.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/RecordOffset.java @@ -73,9 +73,6 @@ public byte[] getRecord() { @Override public String offsetToJSON() { return String - .format( - "{\"topic\":\"%s\", \"partition\":\"%s\", \"offset\":\"%s\"}", this.topic, this.partition, - this.offset - ); + .format("{\"topic\":\"%s\", \"partition\":%d, \"offset\":%d}", this.topic, this.partition, this.offset); } } diff --git a/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java b/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java index e5a4155b..e5124f11 100644 --- a/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java +++ b/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java @@ -152,7 +152,7 @@ public void readCoordinatorTest2Threads() { recordOffset = messages.get(0).get(counter); Assertions .assertEquals( - "{\"topic\":\"testConsumerTopic\", \"partition\":\"7\", \"offset\":\"" + i + "\"}", + "{\"topic\":\"testConsumerTopic\", \"partition\":7, \"offset\":" + i + "}", recordOffset.offsetToJSON() ); rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); @@ -168,7 +168,7 @@ public void readCoordinatorTest2Threads() { recordOffset = messages.get(0).get(counter); Assertions .assertEquals( - "{\"topic\":\"testConsumerTopic\", \"partition\":\"5\", \"offset\":\"" + i + "\"}", + "{\"topic\":\"testConsumerTopic\", \"partition\":5, \"offset\":" + i + "}", recordOffset.offsetToJSON() ); rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); @@ -184,7 +184,7 @@ public void readCoordinatorTest2Threads() { recordOffset = messages.get(0).get(counter); Assertions .assertEquals( - "{\"topic\":\"testConsumerTopic\", \"partition\":\"3\", \"offset\":\"" + i + "\"}", + "{\"topic\":\"testConsumerTopic\", \"partition\":3, \"offset\":" + i + "}", recordOffset.offsetToJSON() ); rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); @@ -200,7 +200,7 @@ public void readCoordinatorTest2Threads() { recordOffset = messages.get(0).get(counter); Assertions .assertEquals( - "{\"topic\":\"testConsumerTopic\", \"partition\":\"1\", \"offset\":\"" + i + "\"}", + "{\"topic\":\"testConsumerTopic\", \"partition\":1, \"offset\":" + i + "}", recordOffset.offsetToJSON() ); rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); @@ -216,7 +216,7 @@ public void readCoordinatorTest2Threads() { recordOffset = messages.get(0).get(counter); Assertions .assertEquals( - "{\"topic\":\"testConsumerTopic\", \"partition\":\"9\", \"offset\":\"" + i + "\"}", + "{\"topic\":\"testConsumerTopic\", \"partition\":9, \"offset\":" + i + "}", recordOffset.offsetToJSON() ); rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); @@ -235,7 +235,7 @@ public void readCoordinatorTest2Threads() { recordOffset = messages.get(1).get(counter); Assertions .assertEquals( - "{\"topic\":\"testConsumerTopic\", \"partition\":\"8\", \"offset\":\"" + i + "\"}", + "{\"topic\":\"testConsumerTopic\", \"partition\":8, \"offset\":" + i + "}", recordOffset.offsetToJSON() ); rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); @@ -251,7 +251,7 @@ public void readCoordinatorTest2Threads() { recordOffset = messages.get(1).get(counter); Assertions .assertEquals( - "{\"topic\":\"testConsumerTopic\", \"partition\":\"6\", \"offset\":\"" + i + "\"}", + "{\"topic\":\"testConsumerTopic\", \"partition\":6, \"offset\":" + i + "}", recordOffset.offsetToJSON() ); rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); @@ -267,7 +267,7 @@ public void readCoordinatorTest2Threads() { recordOffset = messages.get(1).get(counter); Assertions .assertEquals( - "{\"topic\":\"testConsumerTopic\", \"partition\":\"4\", \"offset\":\"" + i + "\"}", + "{\"topic\":\"testConsumerTopic\", \"partition\":4, \"offset\":" + i + "}", recordOffset.offsetToJSON() ); rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); @@ -283,7 +283,7 @@ public void readCoordinatorTest2Threads() { recordOffset = messages.get(1).get(counter); Assertions .assertEquals( - "{\"topic\":\"testConsumerTopic\", \"partition\":\"2\", \"offset\":\"" + i + "\"}", + "{\"topic\":\"testConsumerTopic\", \"partition\":2, \"offset\":" + i + "}", recordOffset.offsetToJSON() ); rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); @@ -299,7 +299,7 @@ public void readCoordinatorTest2Threads() { recordOffset = messages.get(1).get(counter); Assertions .assertEquals( - "{\"topic\":\"testConsumerTopic\", \"partition\":\"0\", \"offset\":\"" + i + "\"}", + "{\"topic\":\"testConsumerTopic\", \"partition\":0, \"offset\":" + i + "}", recordOffset.offsetToJSON() ); rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); @@ -389,7 +389,7 @@ public void readCoordinatorTest1Thread() { recordOffset = messages.get(0).get(counter); Assertions .assertEquals( - "{\"topic\":\"testConsumerTopic\", \"partition\":\"7\", \"offset\":\"" + i + "\"}", + "{\"topic\":\"testConsumerTopic\", \"partition\":7, \"offset\":" + i + "}", recordOffset.offsetToJSON() ); rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); @@ -405,7 +405,7 @@ public void readCoordinatorTest1Thread() { recordOffset = messages.get(0).get(counter); Assertions .assertEquals( - "{\"topic\":\"testConsumerTopic\", \"partition\":\"8\", \"offset\":\"" + i + "\"}", + "{\"topic\":\"testConsumerTopic\", \"partition\":8, \"offset\":" + i + "}", recordOffset.offsetToJSON() ); rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); @@ -421,7 +421,7 @@ public void readCoordinatorTest1Thread() { recordOffset = messages.get(0).get(counter); Assertions .assertEquals( - "{\"topic\":\"testConsumerTopic\", \"partition\":\"5\", \"offset\":\"" + i + "\"}", + "{\"topic\":\"testConsumerTopic\", \"partition\":5, \"offset\":" + i + "}", recordOffset.offsetToJSON() ); rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); @@ -436,7 +436,7 @@ public void readCoordinatorTest1Thread() { recordOffset = messages.get(0).get(counter); Assertions .assertEquals( - "{\"topic\":\"testConsumerTopic\", \"partition\":\"6\", \"offset\":\"" + i + "\"}", + "{\"topic\":\"testConsumerTopic\", \"partition\":6, \"offset\":" + i + "}", recordOffset.offsetToJSON() ); rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); @@ -451,7 +451,7 @@ public void readCoordinatorTest1Thread() { recordOffset = messages.get(0).get(counter); Assertions .assertEquals( - "{\"topic\":\"testConsumerTopic\", \"partition\":\"3\", \"offset\":\"" + i + "\"}", + "{\"topic\":\"testConsumerTopic\", \"partition\":3, \"offset\":" + i + "}", recordOffset.offsetToJSON() ); rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); @@ -466,7 +466,7 @@ public void readCoordinatorTest1Thread() { recordOffset = messages.get(0).get(counter); Assertions .assertEquals( - "{\"topic\":\"testConsumerTopic\", \"partition\":\"4\", \"offset\":\"" + i + "\"}", + "{\"topic\":\"testConsumerTopic\", \"partition\":4, \"offset\":" + i + "}", recordOffset.offsetToJSON() ); rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); @@ -481,7 +481,7 @@ public void readCoordinatorTest1Thread() { recordOffset = messages.get(0).get(counter); Assertions .assertEquals( - "{\"topic\":\"testConsumerTopic\", \"partition\":\"1\", \"offset\":\"" + i + "\"}", + "{\"topic\":\"testConsumerTopic\", \"partition\":1, \"offset\":" + i + "}", recordOffset.offsetToJSON() ); rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); @@ -496,7 +496,7 @@ public void readCoordinatorTest1Thread() { recordOffset = messages.get(0).get(counter); Assertions .assertEquals( - "{\"topic\":\"testConsumerTopic\", \"partition\":\"2\", \"offset\":\"" + i + "\"}", + "{\"topic\":\"testConsumerTopic\", \"partition\":2, \"offset\":" + i + "}", recordOffset.offsetToJSON() ); rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); @@ -511,7 +511,7 @@ public void readCoordinatorTest1Thread() { recordOffset = messages.get(0).get(counter); Assertions .assertEquals( - "{\"topic\":\"testConsumerTopic\", \"partition\":\"0\", \"offset\":\"" + i + "\"}", + "{\"topic\":\"testConsumerTopic\", \"partition\":0, \"offset\":" + i + "}", recordOffset.offsetToJSON() ); rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); @@ -526,7 +526,7 @@ public void readCoordinatorTest1Thread() { recordOffset = messages.get(0).get(counter); Assertions .assertEquals( - "{\"topic\":\"testConsumerTopic\", \"partition\":\"9\", \"offset\":\"" + i + "\"}", + "{\"topic\":\"testConsumerTopic\", \"partition\":9, \"offset\":" + i + "}", recordOffset.offsetToJSON() ); rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); From 3458ee236553e07194a43a2ecd9117789abd040c Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Wed, 26 Jun 2024 08:34:58 +0300 Subject: [PATCH 114/146] Added missing public visibilities. --- .../teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java index 20a14e87..3b893d41 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/SyslogAvroWriter.java @@ -57,7 +57,7 @@ import java.io.*; -class SyslogAvroWriter implements AutoCloseable { +public class SyslogAvroWriter implements AutoCloseable { private static final Logger LOGGER = LoggerFactory.getLogger(SyslogAvroWriter.class); @@ -67,7 +67,7 @@ class SyslogAvroWriter implements AutoCloseable { private final DataFileWriter dataFileWriter = new DataFileWriter<>(datumWriter); - SyslogAvroWriter(File syslogFile) throws IOException { + public SyslogAvroWriter(File syslogFile) throws IOException { dataFileWriter.setCodec(CodecFactory.snappyCodec()); syncableFileOutputStream = new SyncableFileOutputStream(syslogFile); @@ -88,7 +88,7 @@ class SyslogAvroWriter implements AutoCloseable { } } - void write(SyslogRecord syslogRecord) throws IOException { + public void write(SyslogRecord syslogRecord) throws IOException { dataFileWriter.append(syslogRecord); dataFileWriter.flush(); // getFileSize() doesn't work properly if dataFileWriter.flush() is not called after appending a new record to the AVRO-file. From 1ac75a34f16a9de3aa83162015a32c617f56ec47 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Wed, 26 Jun 2024 09:50:33 +0300 Subject: [PATCH 115/146] Removed unneeded concatenation. Renamed single letter variable. --- .../cfe_39/consumers/kafka/queue/WritableQueue.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/queue/WritableQueue.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/queue/WritableQueue.java index ecd32118..4db4116e 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/queue/WritableQueue.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/queue/WritableQueue.java @@ -71,10 +71,10 @@ public WritableQueue(String queueDirectory, String queueNamePrefix) { this.queueDirectory = Paths.get(queueDirectory); this.queueNamePrefix = queueNamePrefix; if (!Files.isDirectory(this.queueDirectory)) { - throw new IllegalArgumentException("Provided path is not a " + "directory <[" + queueDirectory + "]>"); + throw new IllegalArgumentException("Provided path is not a directory <[" + queueDirectory + "]>"); } if (!Files.isWritable(this.queueDirectory)) { - throw new IllegalArgumentException("Provided path is not " + "writeable <[" + queueDirectory + "]>"); + throw new IllegalArgumentException("Provided path is not writeable <[" + queueDirectory + "]>"); } } @@ -108,8 +108,8 @@ public File getNextWritableFile() throws IOException { } } - public void setQueueNamePrefix(String a) { - this.queueNamePrefix = a; + public void setQueueNamePrefix(String queueNamePrefix) { + this.queueNamePrefix = queueNamePrefix; } private BiPredicate getFileMatcher(String queueNamePrefix) { From 75ff318e70d0d66b19c06fe975dc10f65f66fde1 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Wed, 26 Jun 2024 11:18:49 +0300 Subject: [PATCH 116/146] Fixed logging. --- src/test/java/com/teragrep/cfe_39/ConfigTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/test/java/com/teragrep/cfe_39/ConfigTest.java b/src/test/java/com/teragrep/cfe_39/ConfigTest.java index 2f3607d1..b8c41dc3 100644 --- a/src/test/java/com/teragrep/cfe_39/ConfigTest.java +++ b/src/test/java/com/teragrep/cfe_39/ConfigTest.java @@ -69,7 +69,7 @@ public void configTest() { boolean useMockKafkaConsumer = Boolean .parseBoolean(readerKafkaProperties.getProperty("useMockKafkaConsumer", "false")); Assertions.assertTrue(useMockKafkaConsumer); - LOGGER.debug("useMockKafkaConsumer: " + useMockKafkaConsumer); + LOGGER.debug("useMockKafkaConsumer: {}", useMockKafkaConsumer); }); } } From f272ed381fac42c2b8d1c883d4c0b528eeaa312e Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Wed, 26 Jun 2024 14:53:49 +0300 Subject: [PATCH 117/146] Condensed assertions using nested loops. --- .../teragrep/cfe_39/KafkaConsumerTest.java | 181 +++--------------- 1 file changed, 29 insertions(+), 152 deletions(-) diff --git a/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java b/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java index e5124f11..43d5e36b 100644 --- a/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java +++ b/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java @@ -380,160 +380,37 @@ public void readCoordinatorTest1Thread() { ); RFC5424Frame rfc5424Frame = new RFC5424Frame(false); - RecordOffset recordOffset; - - Iterator iterator = list.iterator(); + Iterator iterator; + List partitionList = new ArrayList(); + partitionList.add(7); + partitionList.add(8); + partitionList.add(5); + partitionList.add(6); + partitionList.add(3); + partitionList.add(4); + partitionList.add(1); + partitionList.add(2); + partitionList.add(0); + partitionList.add(9); int counter = 0; - for (int i = 0; i <= 13; i++) { - recordOffset = messages.get(0).get(counter); - Assertions - .assertEquals( - "{\"topic\":\"testConsumerTopic\", \"partition\":7, \"offset\":" + i + "}", - recordOffset.offsetToJSON() - ); - rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); - Assertions.assertTrue(rfc5424Frame.next()); - Assertions.assertTrue(iterator.hasNext()); - Assertions.assertEquals(iterator.next(), rfc5424Frame.msg.toString()); - Assertions.assertFalse(rfc5424Frame.next()); - counter++; - } - - iterator = list.iterator(); - for (int i = 0; i <= 13; i++) { - recordOffset = messages.get(0).get(counter); - Assertions - .assertEquals( - "{\"topic\":\"testConsumerTopic\", \"partition\":8, \"offset\":" + i + "}", - recordOffset.offsetToJSON() - ); - rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); - Assertions.assertTrue(rfc5424Frame.next()); - Assertions.assertTrue(iterator.hasNext()); - Assertions.assertEquals(iterator.next(), rfc5424Frame.msg.toString()); - Assertions.assertFalse(rfc5424Frame.next()); - counter++; - } - - iterator = list.iterator(); - for (int i = 0; i <= 13; i++) { - recordOffset = messages.get(0).get(counter); - Assertions - .assertEquals( - "{\"topic\":\"testConsumerTopic\", \"partition\":5, \"offset\":" + i + "}", - recordOffset.offsetToJSON() - ); - rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); - Assertions.assertTrue(rfc5424Frame.next()); - Assertions.assertEquals(iterator.next(), rfc5424Frame.msg.toString()); - Assertions.assertFalse(rfc5424Frame.next()); - counter++; - } - - iterator = list.iterator(); - for (int i = 0; i <= 13; i++) { - recordOffset = messages.get(0).get(counter); - Assertions - .assertEquals( - "{\"topic\":\"testConsumerTopic\", \"partition\":6, \"offset\":" + i + "}", - recordOffset.offsetToJSON() - ); - rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); - Assertions.assertTrue(rfc5424Frame.next()); - Assertions.assertEquals(iterator.next(), rfc5424Frame.msg.toString()); - Assertions.assertFalse(rfc5424Frame.next()); - counter++; - } - - iterator = list.iterator(); - for (int i = 0; i <= 13; i++) { - recordOffset = messages.get(0).get(counter); - Assertions - .assertEquals( - "{\"topic\":\"testConsumerTopic\", \"partition\":3, \"offset\":" + i + "}", - recordOffset.offsetToJSON() - ); - rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); - Assertions.assertTrue(rfc5424Frame.next()); - Assertions.assertEquals(iterator.next(), rfc5424Frame.msg.toString()); - Assertions.assertFalse(rfc5424Frame.next()); - counter++; - } - - iterator = list.iterator(); - for (int i = 0; i <= 13; i++) { - recordOffset = messages.get(0).get(counter); - Assertions - .assertEquals( - "{\"topic\":\"testConsumerTopic\", \"partition\":4, \"offset\":" + i + "}", - recordOffset.offsetToJSON() - ); - rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); - Assertions.assertTrue(rfc5424Frame.next()); - Assertions.assertEquals(iterator.next(), rfc5424Frame.msg.toString()); - Assertions.assertFalse(rfc5424Frame.next()); - counter++; - } - - iterator = list.iterator(); - for (int i = 0; i <= 13; i++) { - recordOffset = messages.get(0).get(counter); - Assertions - .assertEquals( - "{\"topic\":\"testConsumerTopic\", \"partition\":1, \"offset\":" + i + "}", - recordOffset.offsetToJSON() - ); - rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); - Assertions.assertTrue(rfc5424Frame.next()); - Assertions.assertEquals(iterator.next(), rfc5424Frame.msg.toString()); - Assertions.assertFalse(rfc5424Frame.next()); - counter++; - } - - iterator = list.iterator(); - for (int i = 0; i <= 13; i++) { - recordOffset = messages.get(0).get(counter); - Assertions - .assertEquals( - "{\"topic\":\"testConsumerTopic\", \"partition\":2, \"offset\":" + i + "}", - recordOffset.offsetToJSON() - ); - rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); - Assertions.assertTrue(rfc5424Frame.next()); - Assertions.assertEquals(iterator.next(), rfc5424Frame.msg.toString()); - Assertions.assertFalse(rfc5424Frame.next()); - counter++; - } - - iterator = list.iterator(); - for (int i = 0; i <= 13; i++) { - recordOffset = messages.get(0).get(counter); - Assertions - .assertEquals( - "{\"topic\":\"testConsumerTopic\", \"partition\":0, \"offset\":" + i + "}", - recordOffset.offsetToJSON() - ); - rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); - Assertions.assertTrue(rfc5424Frame.next()); - Assertions.assertEquals(iterator.next(), rfc5424Frame.msg.toString()); - Assertions.assertFalse(rfc5424Frame.next()); - counter++; - } - - iterator = list.iterator(); - for (int i = 0; i <= 13; i++) { - recordOffset = messages.get(0).get(counter); - Assertions - .assertEquals( - "{\"topic\":\"testConsumerTopic\", \"partition\":9, \"offset\":" + i + "}", - recordOffset.offsetToJSON() - ); - rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); - Assertions.assertTrue(rfc5424Frame.next()); - Assertions.assertEquals(iterator.next(), rfc5424Frame.msg.toString()); - Assertions.assertFalse(rfc5424Frame.next()); - counter++; + for (int partition : partitionList) { + iterator = list.iterator(); + for (int i = 0; i <= 13; i++) { + recordOffset = messages.get(0).get(counter); + Assertions + .assertEquals( + "{\"topic\":\"testConsumerTopic\", \"partition\":" + partition + ", \"offset\":" + i + + "}", + recordOffset.offsetToJSON() + ); + rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); + Assertions.assertTrue(rfc5424Frame.next()); + Assertions.assertTrue(iterator.hasNext()); + Assertions.assertEquals(iterator.next(), rfc5424Frame.msg.toString()); + Assertions.assertFalse(rfc5424Frame.next()); + counter++; + } } Assertions.assertEquals(140, counter); // All 140 records asserted. From e57fc1f4e4b45fb6c5874f0b4f3914325c549d14 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Wed, 26 Jun 2024 16:03:45 +0300 Subject: [PATCH 118/146] Improved commenting on tests. --- src/test/java/com/teragrep/cfe_39/HdfsTest.java | 4 ++-- .../java/com/teragrep/cfe_39/Ingestion0FilesTest.java | 8 ++++++-- .../com/teragrep/cfe_39/Ingestion1Old1NewFileTest.java | 6 ++++-- .../java/com/teragrep/cfe_39/Ingestion2NewFilesTest.java | 6 ++++-- .../java/com/teragrep/cfe_39/Ingestion2OldFilesTest.java | 5 +++-- src/test/java/com/teragrep/cfe_39/PruningNoFilesTest.java | 1 + .../java/com/teragrep/cfe_39/PruningOneNewFileTest.java | 2 +- .../java/com/teragrep/cfe_39/PruningOneOldFileTest.java | 2 +- .../com/teragrep/cfe_39/PruningOneOldOneNewFileTest.java | 3 ++- .../java/com/teragrep/cfe_39/PruningTwoNewFilesTest.java | 2 +- .../java/com/teragrep/cfe_39/PruningTwoOldFilesTest.java | 2 +- 11 files changed, 26 insertions(+), 15 deletions(-) diff --git a/src/test/java/com/teragrep/cfe_39/HdfsTest.java b/src/test/java/com/teragrep/cfe_39/HdfsTest.java index 96dad864..33f9cd1e 100644 --- a/src/test/java/com/teragrep/cfe_39/HdfsTest.java +++ b/src/test/java/com/teragrep/cfe_39/HdfsTest.java @@ -116,7 +116,7 @@ public void teardownMiniCluster() { @Test public void hdfsWriteTest() { - // Tests HDFSWrite.java functionality by committing pre-generated AVRO-files to HDFS and assert if it worked as expected. + // This test case is for testing the functionality of the HDFSWrite.java by writing pre-generated AVRO-files to the HDFS database and asserting the results are correct. assertDoesNotThrow(() -> { Assertions.assertFalse(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic"))); @@ -163,7 +163,7 @@ public void hdfsWriteTest() { @Test public void hdfsWriteExceptionTest() { - // File already exists exception test, commits the same file twice to trigger the exception. + // This test case is for testing the functionality of the HDFSWrite.java exception handling by trying to write the same file twice and asserting that the proper exception is thrown. assertDoesNotThrow(() -> { Assertions.assertFalse(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic"))); diff --git a/src/test/java/com/teragrep/cfe_39/Ingestion0FilesTest.java b/src/test/java/com/teragrep/cfe_39/Ingestion0FilesTest.java index 21159b72..f2c311bc 100644 --- a/src/test/java/com/teragrep/cfe_39/Ingestion0FilesTest.java +++ b/src/test/java/com/teragrep/cfe_39/Ingestion0FilesTest.java @@ -114,7 +114,9 @@ public void teardownMiniCluster() { @Test public void ingestion0FilesTest() { - // Empty HDFS database, 140 records in mock kafka consumer ready for ingestion. All 14 records for each 10 topic partitions are stored in a single avro-file per partition. + /*This test case is for testing the functionality of the ingestion when there are no files already present in the database before starting ingestion. + Maximum file size is set to 30,000 in the config. + Empty HDFS database, 140 records in mock kafka consumer ready for ingestion. All 14 records for each 10 topic partitions are stored in a single avro-file per partition.*/ assertDoesNotThrow(() -> { Assertions.assertTrue(config.getPruneOffset() >= 300000L); // Fails the test if the config is not correct. config.setMaximumFileSize(30000); // This parameter defines the amount of records that can fit inside a single AVRO-file. @@ -329,7 +331,9 @@ record = reader.next(record); @Test public void ingestion0FilesLowSizeTest() { - // Empty HDFS database, 140 records in mock kafka consumer ready for ingestion. All 14 records for each 10 topic partitions are stored in two avro-files per partition based on MaximumFileSize. + /*This test case is for testing the functionality of the ingestion when there are files already present in the database before starting ingestion. + Maximum file size is set to 3,000 in the config. + Empty HDFS database, 140 records in mock kafka consumer ready for ingestion. All 14 records for each 10 topic partitions are stored in two avro-files per partition based on MaximumFileSize.*/ assertDoesNotThrow(() -> { Assertions.assertTrue(config.getPruneOffset() >= 300000L); // Fails the test if the config is not correct. config.setMaximumFileSize(3000); // This parameter defines the amount of records that can fit inside a single AVRO-file. diff --git a/src/test/java/com/teragrep/cfe_39/Ingestion1Old1NewFileTest.java b/src/test/java/com/teragrep/cfe_39/Ingestion1Old1NewFileTest.java index c588905b..7b719695 100644 --- a/src/test/java/com/teragrep/cfe_39/Ingestion1Old1NewFileTest.java +++ b/src/test/java/com/teragrep/cfe_39/Ingestion1Old1NewFileTest.java @@ -147,9 +147,11 @@ public void teardownMiniCluster() { @Test public void ingestion1Old1NewFileTest() { - /* 14 records are inserted to HDFS database before starting ingestion, with 124/140 records in mock kafka consumer ready for ingestion. + /* This test case is for testing the functionality of the ingestion when there are files already present in the database before starting ingestion. + 14 records are inserted to HDFS database before starting ingestion, with 124/140 records in mock kafka consumer ready for ingestion. Partitions through 1 to 9 will have only a single file, partition 0 will have 2 files (0.9 and 0.13). - partition 0 files are pre-made and inserted to the HDFS database with old timestamp for file 0.9 and new for 0.13.*/ + partition 0 files are pre-made and inserted to the HDFS database with old timestamp for file 0.9 and new for 0.13. + Old files are pruned from the database during ingestion topic scan loops.*/ assertDoesNotThrow(() -> { // Assert the known starting state. diff --git a/src/test/java/com/teragrep/cfe_39/Ingestion2NewFilesTest.java b/src/test/java/com/teragrep/cfe_39/Ingestion2NewFilesTest.java index 3c04dddd..e8eeb547 100644 --- a/src/test/java/com/teragrep/cfe_39/Ingestion2NewFilesTest.java +++ b/src/test/java/com/teragrep/cfe_39/Ingestion2NewFilesTest.java @@ -148,8 +148,10 @@ public void teardownMiniCluster() { @Test public void ingestion2NewFilesTest() { - // 14 records are inserted to HDFS database before starting ingestion, with 124/140 records in mock kafka consumer ready for ingestion. Partitions through 1 to 9 will have only a single file, partition 0 will have 2 files (0.9 and 0.13). - + /* This test case is for testing the functionality of the ingestion when there are files already present in the database before starting ingestion. + 14 records are inserted to HDFS database before starting ingestion, with 124/140 records in mock kafka consumer ready for ingestion. + Partitions through 1 to 9 will have only a single file, partition 0 will have 2 files (0.9 and 0.13) that are inserted to the database before starting ingestion. + */ assertDoesNotThrow(() -> { // Assert the known starting state. Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic"))); diff --git a/src/test/java/com/teragrep/cfe_39/Ingestion2OldFilesTest.java b/src/test/java/com/teragrep/cfe_39/Ingestion2OldFilesTest.java index 9b84670b..8728b94a 100644 --- a/src/test/java/com/teragrep/cfe_39/Ingestion2OldFilesTest.java +++ b/src/test/java/com/teragrep/cfe_39/Ingestion2OldFilesTest.java @@ -148,8 +148,9 @@ public void teardownMiniCluster() { @Test public void ingestion2OldFilesTest() { - /* 14 records are inserted to HDFS database before starting ingestion, with 126/140 records in mock kafka consumer ready for ingestion. - Partitions through 1 to 9 will have only a single file, partition 0 will have 2 files (0.9 and 0.13). + /* This test case is for testing the functionality of the ingestion when there are files already present in the database before starting ingestion. + 14 records are inserted to HDFS database before starting ingestion, with 126/140 records in mock kafka consumer ready for ingestion. + Partitions through 1 to 9 will have only a single file, partition 0 will have 2 files (0.9 and 0.13) that are inserted to the database before starting ingestion. partition 0 files are pre-made and inserted to the HDFS database with old timestamps that will mark them for pruning when ingestion is started.*/ assertDoesNotThrow(() -> { diff --git a/src/test/java/com/teragrep/cfe_39/PruningNoFilesTest.java b/src/test/java/com/teragrep/cfe_39/PruningNoFilesTest.java index a3472544..29370b7c 100644 --- a/src/test/java/com/teragrep/cfe_39/PruningNoFilesTest.java +++ b/src/test/java/com/teragrep/cfe_39/PruningNoFilesTest.java @@ -113,6 +113,7 @@ public void teardownMiniCluster() { @Test public void noFiles() { + // This test case is for testing the functionality of the HDFSPrune.java when the target database is empty. assertDoesNotThrow(() -> { Assertions.assertFalse(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic"))); HDFSPrune hdfsPrune = new HDFSPrune(config, "testConsumerTopic", fs); diff --git a/src/test/java/com/teragrep/cfe_39/PruningOneNewFileTest.java b/src/test/java/com/teragrep/cfe_39/PruningOneNewFileTest.java index 8b106c8a..d915b319 100644 --- a/src/test/java/com/teragrep/cfe_39/PruningOneNewFileTest.java +++ b/src/test/java/com/teragrep/cfe_39/PruningOneNewFileTest.java @@ -144,7 +144,7 @@ public void teardownMiniCluster() { @Test public void oneNewFileTest() { - // Test for not triggering pruning for a single file. + // This test case is for testing the functionality of the HDFSPrune.java when the database holds a file with a timestamp that shouldn't trigger pruning of old files. Assertions.assertTrue(config.getPruneOffset() >= 300000L); // Fails the test if the config is not correct, too low pruning offset can prune the files if the test is lagging. Assertions.assertTrue(System.currentTimeMillis() - config.getPruneOffset() > 157784760000L); diff --git a/src/test/java/com/teragrep/cfe_39/PruningOneOldFileTest.java b/src/test/java/com/teragrep/cfe_39/PruningOneOldFileTest.java index eefc42da..7a44a247 100644 --- a/src/test/java/com/teragrep/cfe_39/PruningOneOldFileTest.java +++ b/src/test/java/com/teragrep/cfe_39/PruningOneOldFileTest.java @@ -144,7 +144,7 @@ public void teardownMiniCluster() { @Test public void oneOldFileTest() { - // Test for triggering pruning for a single file. + // This test case is for testing the functionality of the HDFSPrune.java when the database holds a file with a timestamp that should trigger pruning of old files. Assertions.assertTrue(config.getPruneOffset() >= 300000L); // Fails the test if the config is not correct, too low pruning offset can prune the files if the test is lagging. Assertions.assertTrue(System.currentTimeMillis() - config.getPruneOffset() > 157784760000L); diff --git a/src/test/java/com/teragrep/cfe_39/PruningOneOldOneNewFileTest.java b/src/test/java/com/teragrep/cfe_39/PruningOneOldOneNewFileTest.java index e2f9c69f..41b21b7d 100644 --- a/src/test/java/com/teragrep/cfe_39/PruningOneOldOneNewFileTest.java +++ b/src/test/java/com/teragrep/cfe_39/PruningOneOldOneNewFileTest.java @@ -147,7 +147,8 @@ public void teardownMiniCluster() { @Test public void oneOldOneNewFileTest() { - // Test for triggering pruning for 1 out of 2 files in the topic. + /* This test case is for testing the functionality of the HDFSPrune.java when the database holds a file with a timestamp that shouldn't trigger pruning of old files and another file that should trigger the pruning. + The file with newer timestamp is ignored while the older is deleted from the database.*/ Assertions.assertTrue(config.getPruneOffset() >= 300000L); // Fails the test if the config is not correct, too low pruning offset can prune the files if the test is lagging. Assertions.assertTrue(System.currentTimeMillis() - config.getPruneOffset() > 157784760000L); diff --git a/src/test/java/com/teragrep/cfe_39/PruningTwoNewFilesTest.java b/src/test/java/com/teragrep/cfe_39/PruningTwoNewFilesTest.java index 0e042bb3..d3ef50fe 100644 --- a/src/test/java/com/teragrep/cfe_39/PruningTwoNewFilesTest.java +++ b/src/test/java/com/teragrep/cfe_39/PruningTwoNewFilesTest.java @@ -147,7 +147,7 @@ public void teardownMiniCluster() { @Test public void twoNewFilesTest() { - // Test for not triggering pruning for 2 files in the topic. + // This test case is for testing the functionality of the HDFSPrune.java when the database holds two files with a timestamp that shouldn't trigger pruning of old files. Assertions.assertTrue(config.getPruneOffset() >= 300000L); // Fails the test if the config is not correct, too low pruning offset can prune the files if the test is lagging. assertDoesNotThrow(() -> { diff --git a/src/test/java/com/teragrep/cfe_39/PruningTwoOldFilesTest.java b/src/test/java/com/teragrep/cfe_39/PruningTwoOldFilesTest.java index 1f0b8624..ca50b7dc 100644 --- a/src/test/java/com/teragrep/cfe_39/PruningTwoOldFilesTest.java +++ b/src/test/java/com/teragrep/cfe_39/PruningTwoOldFilesTest.java @@ -146,7 +146,7 @@ public void teardownMiniCluster() { @Test public void twoOldFilesTest() { - // Test for triggering pruning for 2 files in the topic. + // This test case is for testing the functionality of the HDFSPrune.java when the database holds two files with a timestamp that should trigger pruning of old files. Assertions.assertTrue(config.getPruneOffset() >= 300000L); // Fails the test if the config is not correct, too low pruning offset can prune the files if the test is lagging. Assertions.assertTrue(System.currentTimeMillis() - config.getPruneOffset() > 157784760000L); From 053476ea6c9f32ff764470b24805d363d86905d6 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Thu, 27 Jun 2024 10:05:56 +0300 Subject: [PATCH 119/146] Removed now unneeded additional FileSystem initializations from ingestion tests. --- .../teragrep/cfe_39/Ingestion0FilesTest.java | 20 +------------------ .../cfe_39/Ingestion1Old1NewFileTest.java | 15 -------------- .../cfe_39/Ingestion2NewFilesTest.java | 15 -------------- .../cfe_39/Ingestion2OldFilesTest.java | 15 -------------- 4 files changed, 1 insertion(+), 64 deletions(-) diff --git a/src/test/java/com/teragrep/cfe_39/Ingestion0FilesTest.java b/src/test/java/com/teragrep/cfe_39/Ingestion0FilesTest.java index f2c311bc..6a228c67 100644 --- a/src/test/java/com/teragrep/cfe_39/Ingestion0FilesTest.java +++ b/src/test/java/com/teragrep/cfe_39/Ingestion0FilesTest.java @@ -127,26 +127,8 @@ public void ingestion0FilesTest() { }); // Assert that the kafka records were ingested correctly and the database holds the correct 140 records. - - // Check that the files were properly written to HDFS. - String hdfsuri = config.getHdfsuri(); - - String path = config.getHdfsPath() + "/" + "testConsumerTopic"; - // ====== Init HDFS File System Object - Configuration conf = new Configuration(); - // Set FileSystem URI - conf.set("fs.defaultFS", hdfsuri); - // Because of Maven - conf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); - conf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); - // Set HADOOP user - System.setProperty("HADOOP_USER_NAME", "hdfs"); - System.setProperty("hadoop.home.dir", "/"); - //Get the filesystem - HDFS assertDoesNotThrow(() -> { - fs = FileSystem.get(URI.create(hdfsuri), conf); - - Path workingDir = fs.getWorkingDirectory(); + String path = config.getHdfsPath() + "/" + "testConsumerTopic"; Path newDirectoryPath = new Path(path); Assertions.assertTrue(fs.exists(newDirectoryPath)); diff --git a/src/test/java/com/teragrep/cfe_39/Ingestion1Old1NewFileTest.java b/src/test/java/com/teragrep/cfe_39/Ingestion1Old1NewFileTest.java index 7b719695..f23434e9 100644 --- a/src/test/java/com/teragrep/cfe_39/Ingestion1Old1NewFileTest.java +++ b/src/test/java/com/teragrep/cfe_39/Ingestion1Old1NewFileTest.java @@ -167,21 +167,6 @@ public void ingestion1Old1NewFileTest() { Thread.sleep(10000); hdfsDataIngestion.run(); - // hdfsDataIngestion.run(); has called fs.close() after finishing ingesting the records from kafka. Rebuild fs. - String hdfsURI = "hdfs://localhost:" + hdfsCluster.getNameNodePort() + "/"; - config.setHdfsuri(hdfsURI); - // ====== Init HDFS File System Object - Configuration fsConf = new Configuration(); - // Set FileSystem URI - fsConf.set("fs.defaultFS", hdfsURI); - // Because of Maven - fsConf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); - fsConf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); - // Set HADOOP user - System.setProperty("HADOOP_USER_NAME", "hdfs"); - System.setProperty("hadoop.home.dir", "/"); - fs = FileSystem.get(URI.create(hdfsURI), fsConf); - // Assert that the kafka records were ingested and pruned correctly and the database holds only the expected 10 files. Assertions .assertEquals(10, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); diff --git a/src/test/java/com/teragrep/cfe_39/Ingestion2NewFilesTest.java b/src/test/java/com/teragrep/cfe_39/Ingestion2NewFilesTest.java index e8eeb547..ec6bafae 100644 --- a/src/test/java/com/teragrep/cfe_39/Ingestion2NewFilesTest.java +++ b/src/test/java/com/teragrep/cfe_39/Ingestion2NewFilesTest.java @@ -165,21 +165,6 @@ public void ingestion2NewFilesTest() { Thread.sleep(10000); hdfsDataIngestion.run(); - // hdfsDataIngestion.run(); has called fs.close() after finishing ingesting the records from kafka. Rebuild fs. - String hdfsURI = "hdfs://localhost:" + hdfsCluster.getNameNodePort() + "/"; - config.setHdfsuri(hdfsURI); - // ====== Init HDFS File System Object - Configuration fsConf = new Configuration(); - // Set FileSystem URI - fsConf.set("fs.defaultFS", hdfsURI); - // Because of Maven - fsConf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); - fsConf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); - // Set HADOOP user - System.setProperty("HADOOP_USER_NAME", "hdfs"); - System.setProperty("hadoop.home.dir", "/"); - fs = FileSystem.get(URI.create(hdfsURI), fsConf); - // Assert that the kafka records were ingested correctly and the database holds the expected 11 files holding the expected 140 records. Assertions .assertEquals(11, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); diff --git a/src/test/java/com/teragrep/cfe_39/Ingestion2OldFilesTest.java b/src/test/java/com/teragrep/cfe_39/Ingestion2OldFilesTest.java index 8728b94a..522d5be5 100644 --- a/src/test/java/com/teragrep/cfe_39/Ingestion2OldFilesTest.java +++ b/src/test/java/com/teragrep/cfe_39/Ingestion2OldFilesTest.java @@ -167,21 +167,6 @@ public void ingestion2OldFilesTest() { Thread.sleep(10000); hdfsDataIngestion.run(); - // hdfsDataIngestion.run(); has called fs.close() after finishing ingesting the records from kafka. Rebuild fs. - String hdfsURI = "hdfs://localhost:" + hdfsCluster.getNameNodePort() + "/"; - config.setHdfsuri(hdfsURI); - // ====== Init HDFS File System Object - Configuration fsConf = new Configuration(); - // Set FileSystem URI - fsConf.set("fs.defaultFS", hdfsURI); - // Because of Maven - fsConf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); - fsConf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); - // Set HADOOP user - System.setProperty("HADOOP_USER_NAME", "hdfs"); - System.setProperty("hadoop.home.dir", "/"); - fs = FileSystem.get(URI.create(hdfsURI), fsConf); - // Assert that the kafka records were ingested and pruned correctly and the database holds only the expected 9 files. Assertions .assertEquals(9, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); From 90c027864195de77ae239787dc35be909fa23006 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Thu, 27 Jun 2024 11:26:39 +0300 Subject: [PATCH 120/146] Implemented TestMiniClusterFactory.java and TestFileSystemFactory.java helper classes for making MiniCluster and FileSystem objects in tests. --- .../java/com/teragrep/cfe_39/HdfsTest.java | 24 +------ .../teragrep/cfe_39/Ingestion0FilesTest.java | 22 +----- .../cfe_39/Ingestion1Old1NewFileTest.java | 23 +----- .../cfe_39/Ingestion2NewFilesTest.java | 23 +----- .../cfe_39/Ingestion2OldFilesTest.java | 23 +----- .../teragrep/cfe_39/PruningNoFilesTest.java | 24 +------ .../cfe_39/PruningOneNewFileTest.java | 23 +----- .../cfe_39/PruningOneOldFileTest.java | 23 +----- .../cfe_39/PruningOneOldOneNewFileTest.java | 23 +----- .../cfe_39/PruningTwoNewFilesTest.java | 24 +------ .../cfe_39/PruningTwoOldFilesTest.java | 23 +----- .../cfe_39/TestFileSystemFactory.java | 71 +++++++++++++++++++ .../cfe_39/TestMiniClusterFactory.java | 70 ++++++++++++++++++ 13 files changed, 164 insertions(+), 232 deletions(-) create mode 100644 src/test/java/com/teragrep/cfe_39/TestFileSystemFactory.java create mode 100644 src/test/java/com/teragrep/cfe_39/TestMiniClusterFactory.java diff --git a/src/test/java/com/teragrep/cfe_39/HdfsTest.java b/src/test/java/com/teragrep/cfe_39/HdfsTest.java index 33f9cd1e..423577e7 100644 --- a/src/test/java/com/teragrep/cfe_39/HdfsTest.java +++ b/src/test/java/com/teragrep/cfe_39/HdfsTest.java @@ -48,18 +48,15 @@ import com.google.gson.JsonObject; import com.google.gson.JsonParser; import com.teragrep.cfe_39.consumers.kafka.HDFSWrite; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.junit.jupiter.api.*; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.File; -import java.net.URI; import java.nio.file.Files; import java.nio.file.Paths; @@ -82,25 +79,8 @@ public void startMiniCluster() { config = new Config(); // Create a HDFS miniCluster baseDir = Files.createTempDirectory("test_hdfs").toFile().getAbsoluteFile(); - Configuration conf = new Configuration(); - conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, baseDir.getAbsolutePath()); - MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf); - hdfsCluster = builder.build(); - String hdfsURI = "hdfs://localhost:" + hdfsCluster.getNameNodePort() + "/"; - config.setHdfsuri(hdfsURI); - DistributedFileSystem fileSystem = hdfsCluster.getFileSystem(); - - // ====== Init HDFS File System Object - Configuration fsConf = new Configuration(); - // Set FileSystem URI - fsConf.set("fs.defaultFS", hdfsURI); - // Because of Maven - fsConf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); - fsConf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); - // Set HADOOP user - System.setProperty("HADOOP_USER_NAME", "hdfs"); - System.setProperty("hadoop.home.dir", "/"); - fs = FileSystem.get(URI.create(hdfsURI), fsConf); + hdfsCluster = new TestMiniClusterFactory().create(config, baseDir); + fs = new TestFileSystemFactory().create(config.getHdfsuri()); }); } diff --git a/src/test/java/com/teragrep/cfe_39/Ingestion0FilesTest.java b/src/test/java/com/teragrep/cfe_39/Ingestion0FilesTest.java index 6a228c67..648af468 100644 --- a/src/test/java/com/teragrep/cfe_39/Ingestion0FilesTest.java +++ b/src/test/java/com/teragrep/cfe_39/Ingestion0FilesTest.java @@ -51,7 +51,6 @@ import org.apache.avro.specific.SpecificDatumReader; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.*; -import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.junit.jupiter.api.*; import org.slf4j.Logger; @@ -80,25 +79,8 @@ public void startMiniCluster() { config = new Config(); // Create a HDFS miniCluster baseDir = Files.createTempDirectory("test_hdfs").toFile().getAbsoluteFile(); - Configuration conf = new Configuration(); - conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, baseDir.getAbsolutePath()); - MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf); - hdfsCluster = builder.build(); - String hdfsURI = "hdfs://localhost:" + hdfsCluster.getNameNodePort() + "/"; - config.setHdfsuri(hdfsURI); - DistributedFileSystem fileSystem = hdfsCluster.getFileSystem(); - - // ====== Init HDFS File System Object - Configuration fsConf = new Configuration(); - // Set FileSystem URI - fsConf.set("fs.defaultFS", hdfsURI); - // Because of Maven - fsConf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); - fsConf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); - // Set HADOOP user - System.setProperty("HADOOP_USER_NAME", "hdfs"); - System.setProperty("hadoop.home.dir", "/"); - fs = FileSystem.get(URI.create(hdfsURI), fsConf); + hdfsCluster = new TestMiniClusterFactory().create(config, baseDir); + fs = new TestFileSystemFactory().create(config.getHdfsuri()); }); } diff --git a/src/test/java/com/teragrep/cfe_39/Ingestion1Old1NewFileTest.java b/src/test/java/com/teragrep/cfe_39/Ingestion1Old1NewFileTest.java index f23434e9..548d490c 100644 --- a/src/test/java/com/teragrep/cfe_39/Ingestion1Old1NewFileTest.java +++ b/src/test/java/com/teragrep/cfe_39/Ingestion1Old1NewFileTest.java @@ -46,11 +46,9 @@ package com.teragrep.cfe_39; import com.teragrep.cfe_39.consumers.kafka.HdfsDataIngestion; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Assertions; @@ -60,7 +58,6 @@ import org.slf4j.LoggerFactory; import java.io.File; -import java.net.URI; import java.nio.file.Files; import java.util.Objects; import java.util.Set; @@ -84,25 +81,9 @@ public void startMiniCluster() { config = new Config(); // Create a HDFS miniCluster baseDir = Files.createTempDirectory("test_hdfs").toFile().getAbsoluteFile(); - Configuration conf = new Configuration(); - conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, baseDir.getAbsolutePath()); - MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf); - hdfsCluster = builder.build(); - String hdfsURI = "hdfs://localhost:" + hdfsCluster.getNameNodePort() + "/"; - config.setHdfsuri(hdfsURI); - DistributedFileSystem fileSystem = hdfsCluster.getFileSystem(); + hdfsCluster = new TestMiniClusterFactory().create(config, baseDir); + fs = new TestFileSystemFactory().create(config.getHdfsuri()); - // ====== Init HDFS File System Object - Configuration fsConf = new Configuration(); - // Set FileSystem URI - fsConf.set("fs.defaultFS", hdfsURI); - // Because of Maven - fsConf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); - fsConf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); - // Set HADOOP user - System.setProperty("HADOOP_USER_NAME", "hdfs"); - System.setProperty("hadoop.home.dir", "/"); - fs = FileSystem.get(URI.create(hdfsURI), fsConf); // Inserts pre-made avro-files to HDFS where one file has new timestamp and other old, which are normally generated during data ingestion from mock kafka consumer. String path = config.getHdfsPath() + "/" + "testConsumerTopic"; // "hdfs:///opt/teragrep/cfe_39/srv/testConsumerTopic" // Sets the directory where the data should be stored, if the directory doesn't exist then it's created. diff --git a/src/test/java/com/teragrep/cfe_39/Ingestion2NewFilesTest.java b/src/test/java/com/teragrep/cfe_39/Ingestion2NewFilesTest.java index ec6bafae..7479f1aa 100644 --- a/src/test/java/com/teragrep/cfe_39/Ingestion2NewFilesTest.java +++ b/src/test/java/com/teragrep/cfe_39/Ingestion2NewFilesTest.java @@ -51,7 +51,6 @@ import org.apache.avro.specific.SpecificDatumReader; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.*; -import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Assertions; @@ -87,25 +86,9 @@ public void startMiniCluster() { config = new Config(); // Create a HDFS miniCluster baseDir = Files.createTempDirectory("test_hdfs").toFile().getAbsoluteFile(); - Configuration conf = new Configuration(); - conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, baseDir.getAbsolutePath()); - MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf); - hdfsCluster = builder.build(); - String hdfsURI = "hdfs://localhost:" + hdfsCluster.getNameNodePort() + "/"; - config.setHdfsuri(hdfsURI); - DistributedFileSystem fileSystem = hdfsCluster.getFileSystem(); - - // ====== Init HDFS File System Object - Configuration fsConf = new Configuration(); - // Set FileSystem URI - fsConf.set("fs.defaultFS", hdfsURI); - // Because of Maven - fsConf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); - fsConf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); - // Set HADOOP user - System.setProperty("HADOOP_USER_NAME", "hdfs"); - System.setProperty("hadoop.home.dir", "/"); - fs = FileSystem.get(URI.create(hdfsURI), fsConf); + hdfsCluster = new TestMiniClusterFactory().create(config, baseDir); + fs = new TestFileSystemFactory().create(config.getHdfsuri()); + // Inserts pre-made avro-files with new timestamps to HDFS, which are normally generated during data ingestion from mock kafka consumer. String path = config.getHdfsPath() + "/" + "testConsumerTopic"; // "hdfs:///opt/teragrep/cfe_39/srv/testConsumerTopic" // Sets the directory where the data should be stored, if the directory doesn't exist then it's created. diff --git a/src/test/java/com/teragrep/cfe_39/Ingestion2OldFilesTest.java b/src/test/java/com/teragrep/cfe_39/Ingestion2OldFilesTest.java index 522d5be5..d40895bd 100644 --- a/src/test/java/com/teragrep/cfe_39/Ingestion2OldFilesTest.java +++ b/src/test/java/com/teragrep/cfe_39/Ingestion2OldFilesTest.java @@ -46,11 +46,9 @@ package com.teragrep.cfe_39; import com.teragrep.cfe_39.consumers.kafka.HdfsDataIngestion; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Assertions; @@ -60,7 +58,6 @@ import org.slf4j.LoggerFactory; import java.io.File; -import java.net.URI; import java.nio.file.Files; import java.util.Objects; import java.util.Set; @@ -84,25 +81,9 @@ public void startMiniCluster() { config = new Config(); // Create a HDFS miniCluster baseDir = Files.createTempDirectory("test_hdfs").toFile().getAbsoluteFile(); - Configuration conf = new Configuration(); - conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, baseDir.getAbsolutePath()); - MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf); - hdfsCluster = builder.build(); - String hdfsURI = "hdfs://localhost:" + hdfsCluster.getNameNodePort() + "/"; - config.setHdfsuri(hdfsURI); - DistributedFileSystem fileSystem = hdfsCluster.getFileSystem(); + hdfsCluster = new TestMiniClusterFactory().create(config, baseDir); + fs = new TestFileSystemFactory().create(config.getHdfsuri()); - // ====== Init HDFS File System Object - Configuration fsConf = new Configuration(); - // Set FileSystem URI - fsConf.set("fs.defaultFS", hdfsURI); - // Because of Maven - fsConf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); - fsConf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); - // Set HADOOP user - System.setProperty("HADOOP_USER_NAME", "hdfs"); - System.setProperty("hadoop.home.dir", "/"); - fs = FileSystem.get(URI.create(hdfsURI), fsConf); // Inserts pre-made avro-files with old timestamps to HDFS, which are normally generated during data ingestion from mock kafka consumer. String path = config.getHdfsPath() + "/" + "testConsumerTopic"; // "hdfs:///opt/teragrep/cfe_39/srv/testConsumerTopic" // Sets the directory where the data should be stored, if the directory doesn't exist then it's created. diff --git a/src/test/java/com/teragrep/cfe_39/PruningNoFilesTest.java b/src/test/java/com/teragrep/cfe_39/PruningNoFilesTest.java index 29370b7c..e04b4915 100644 --- a/src/test/java/com/teragrep/cfe_39/PruningNoFilesTest.java +++ b/src/test/java/com/teragrep/cfe_39/PruningNoFilesTest.java @@ -46,11 +46,9 @@ package com.teragrep.cfe_39; import com.teragrep.cfe_39.consumers.kafka.HDFSPrune; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Assertions; @@ -60,7 +58,6 @@ import org.slf4j.LoggerFactory; import java.io.File; -import java.net.URI; import java.nio.file.Files; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; @@ -80,25 +77,8 @@ public void startMiniCluster() { config = new Config(); // Create a HDFS miniCluster baseDir = Files.createTempDirectory("test_hdfs").toFile().getAbsoluteFile(); - Configuration conf = new Configuration(); - conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, baseDir.getAbsolutePath()); - MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf); - hdfsCluster = builder.build(); - String hdfsURI = "hdfs://localhost:" + hdfsCluster.getNameNodePort() + "/"; - config.setHdfsuri(hdfsURI); - DistributedFileSystem fileSystem = hdfsCluster.getFileSystem(); - - // ====== Init HDFS File System Object - Configuration fsConf = new Configuration(); - // Set FileSystem URI - fsConf.set("fs.defaultFS", hdfsURI); - // Because of Maven - fsConf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); - fsConf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); - // Set HADOOP user - System.setProperty("HADOOP_USER_NAME", "hdfs"); - System.setProperty("hadoop.home.dir", "/"); - fs = FileSystem.get(URI.create(hdfsURI), fsConf); + hdfsCluster = new TestMiniClusterFactory().create(config, baseDir); + fs = new TestFileSystemFactory().create(config.getHdfsuri()); }); } diff --git a/src/test/java/com/teragrep/cfe_39/PruningOneNewFileTest.java b/src/test/java/com/teragrep/cfe_39/PruningOneNewFileTest.java index d915b319..7b210586 100644 --- a/src/test/java/com/teragrep/cfe_39/PruningOneNewFileTest.java +++ b/src/test/java/com/teragrep/cfe_39/PruningOneNewFileTest.java @@ -46,11 +46,9 @@ package com.teragrep.cfe_39; import com.teragrep.cfe_39.consumers.kafka.HDFSPrune; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Assertions; @@ -60,7 +58,6 @@ import org.slf4j.LoggerFactory; import java.io.File; -import java.net.URI; import java.nio.file.Files; import java.util.Objects; import java.util.Set; @@ -84,25 +81,9 @@ public void startMiniCluster() { config = new Config(); // Create a HDFS miniCluster baseDir = Files.createTempDirectory("test_hdfs").toFile().getAbsoluteFile(); - Configuration conf = new Configuration(); - conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, baseDir.getAbsolutePath()); - MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf); - hdfsCluster = builder.build(); - String hdfsURI = "hdfs://localhost:" + hdfsCluster.getNameNodePort() + "/"; - config.setHdfsuri(hdfsURI); - DistributedFileSystem fileSystem = hdfsCluster.getFileSystem(); + hdfsCluster = new TestMiniClusterFactory().create(config, baseDir); + fs = new TestFileSystemFactory().create(config.getHdfsuri()); - // ====== Init HDFS File System Object - Configuration fsConf = new Configuration(); - // Set FileSystem URI - fsConf.set("fs.defaultFS", hdfsURI); - // Because of Maven - fsConf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); - fsConf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); - // Set HADOOP user - System.setProperty("HADOOP_USER_NAME", "hdfs"); - System.setProperty("hadoop.home.dir", "/"); - fs = FileSystem.get(URI.create(hdfsURI), fsConf); // Inserts a single pre-made avro-file with a new timestamp to HDFS, which is normally generated during data ingestion from mock kafka consumer. String path = config.getHdfsPath() + "/" + "testConsumerTopic"; // "hdfs:///opt/teragrep/cfe_39/srv/testConsumerTopic" // Sets the directory where the data should be stored, if the directory doesn't exist then it's created. diff --git a/src/test/java/com/teragrep/cfe_39/PruningOneOldFileTest.java b/src/test/java/com/teragrep/cfe_39/PruningOneOldFileTest.java index 7a44a247..1a5c4021 100644 --- a/src/test/java/com/teragrep/cfe_39/PruningOneOldFileTest.java +++ b/src/test/java/com/teragrep/cfe_39/PruningOneOldFileTest.java @@ -46,11 +46,9 @@ package com.teragrep.cfe_39; import com.teragrep.cfe_39.consumers.kafka.HDFSPrune; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Assertions; @@ -60,7 +58,6 @@ import org.slf4j.LoggerFactory; import java.io.File; -import java.net.URI; import java.nio.file.Files; import java.util.Objects; import java.util.Set; @@ -84,25 +81,9 @@ public void startMiniCluster() { config = new Config(); // Create a HDFS miniCluster baseDir = Files.createTempDirectory("test_hdfs").toFile().getAbsoluteFile(); - Configuration conf = new Configuration(); - conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, baseDir.getAbsolutePath()); - MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf); - hdfsCluster = builder.build(); - String hdfsURI = "hdfs://localhost:" + hdfsCluster.getNameNodePort() + "/"; - config.setHdfsuri(hdfsURI); - DistributedFileSystem fileSystem = hdfsCluster.getFileSystem(); + hdfsCluster = new TestMiniClusterFactory().create(config, baseDir); + fs = new TestFileSystemFactory().create(config.getHdfsuri()); - // ====== Init HDFS File System Object - Configuration fsConf = new Configuration(); - // Set FileSystem URI - fsConf.set("fs.defaultFS", hdfsURI); - // Because of Maven - fsConf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); - fsConf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); - // Set HADOOP user - System.setProperty("HADOOP_USER_NAME", "hdfs"); - System.setProperty("hadoop.home.dir", "/"); - fs = FileSystem.get(URI.create(hdfsURI), fsConf); // Inserts a single pre-made avro-file with an olf timestamp to HDFS, which is normally generated during data ingestion from mock kafka consumer. String path = config.getHdfsPath() + "/" + "testConsumerTopic"; // "hdfs:///opt/teragrep/cfe_39/srv/testConsumerTopic" // Sets the directory where the data should be stored, if the directory doesn't exist then it's created. diff --git a/src/test/java/com/teragrep/cfe_39/PruningOneOldOneNewFileTest.java b/src/test/java/com/teragrep/cfe_39/PruningOneOldOneNewFileTest.java index 41b21b7d..6bf1982f 100644 --- a/src/test/java/com/teragrep/cfe_39/PruningOneOldOneNewFileTest.java +++ b/src/test/java/com/teragrep/cfe_39/PruningOneOldOneNewFileTest.java @@ -46,11 +46,9 @@ package com.teragrep.cfe_39; import com.teragrep.cfe_39.consumers.kafka.HDFSPrune; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Assertions; @@ -60,7 +58,6 @@ import org.slf4j.LoggerFactory; import java.io.File; -import java.net.URI; import java.nio.file.Files; import java.util.Objects; import java.util.Set; @@ -84,25 +81,9 @@ public void startMiniCluster() { config = new Config(); // Create a HDFS miniCluster baseDir = Files.createTempDirectory("test_hdfs").toFile().getAbsoluteFile(); - Configuration conf = new Configuration(); - conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, baseDir.getAbsolutePath()); - MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf); - hdfsCluster = builder.build(); - String hdfsURI = "hdfs://localhost:" + hdfsCluster.getNameNodePort() + "/"; - config.setHdfsuri(hdfsURI); - DistributedFileSystem fileSystem = hdfsCluster.getFileSystem(); + hdfsCluster = new TestMiniClusterFactory().create(config, baseDir); + fs = new TestFileSystemFactory().create(config.getHdfsuri()); - // ====== Init HDFS File System Object - Configuration fsConf = new Configuration(); - // Set FileSystem URI - fsConf.set("fs.defaultFS", hdfsURI); - // Because of Maven - fsConf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); - fsConf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); - // Set HADOOP user - System.setProperty("HADOOP_USER_NAME", "hdfs"); - System.setProperty("hadoop.home.dir", "/"); - fs = FileSystem.get(URI.create(hdfsURI), fsConf); /* Inserts pre-made avro-files to HDFS, which are normally generated during data ingestion from mock kafka consumer. One file has new timestamp and another old timestamp.*/ String path = config.getHdfsPath() + "/" + "testConsumerTopic"; // "hdfs:///opt/teragrep/cfe_39/srv/testConsumerTopic" diff --git a/src/test/java/com/teragrep/cfe_39/PruningTwoNewFilesTest.java b/src/test/java/com/teragrep/cfe_39/PruningTwoNewFilesTest.java index d3ef50fe..c79d4377 100644 --- a/src/test/java/com/teragrep/cfe_39/PruningTwoNewFilesTest.java +++ b/src/test/java/com/teragrep/cfe_39/PruningTwoNewFilesTest.java @@ -46,11 +46,9 @@ package com.teragrep.cfe_39; import com.teragrep.cfe_39.consumers.kafka.HDFSPrune; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Assertions; @@ -60,7 +58,6 @@ import org.slf4j.LoggerFactory; import java.io.File; -import java.net.URI; import java.nio.file.Files; import java.util.Objects; import java.util.Set; @@ -84,25 +81,8 @@ public void startMiniCluster() { config = new Config(); // Create a HDFS miniCluster baseDir = Files.createTempDirectory("test_hdfs").toFile().getAbsoluteFile(); - Configuration conf = new Configuration(); - conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, baseDir.getAbsolutePath()); - MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf); - hdfsCluster = builder.build(); - String hdfsURI = "hdfs://localhost:" + hdfsCluster.getNameNodePort() + "/"; - config.setHdfsuri(hdfsURI); - DistributedFileSystem fileSystem = hdfsCluster.getFileSystem(); - - // ====== Init HDFS File System Object - Configuration fsConf = new Configuration(); - // Set FileSystem URI - fsConf.set("fs.defaultFS", hdfsURI); - // Because of Maven - fsConf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); - fsConf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); - // Set HADOOP user - System.setProperty("HADOOP_USER_NAME", "hdfs"); - System.setProperty("hadoop.home.dir", "/"); - fs = FileSystem.get(URI.create(hdfsURI), fsConf); + hdfsCluster = new TestMiniClusterFactory().create(config, baseDir); + fs = new TestFileSystemFactory().create(config.getHdfsuri()); // Inserts pre-made avro-files with new timestamps to HDFS, which are normally generated during data ingestion from mock kafka consumer. diff --git a/src/test/java/com/teragrep/cfe_39/PruningTwoOldFilesTest.java b/src/test/java/com/teragrep/cfe_39/PruningTwoOldFilesTest.java index ca50b7dc..2c570500 100644 --- a/src/test/java/com/teragrep/cfe_39/PruningTwoOldFilesTest.java +++ b/src/test/java/com/teragrep/cfe_39/PruningTwoOldFilesTest.java @@ -46,11 +46,9 @@ package com.teragrep.cfe_39; import com.teragrep.cfe_39.consumers.kafka.HDFSPrune; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Assertions; @@ -60,7 +58,6 @@ import org.slf4j.LoggerFactory; import java.io.File; -import java.net.URI; import java.nio.file.Files; import java.util.Objects; import java.util.Set; @@ -84,25 +81,9 @@ public void startMiniCluster() { config = new Config(); // Create a HDFS miniCluster baseDir = Files.createTempDirectory("test_hdfs").toFile().getAbsoluteFile(); - Configuration conf = new Configuration(); - conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, baseDir.getAbsolutePath()); - MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf); - hdfsCluster = builder.build(); - String hdfsURI = "hdfs://localhost:" + hdfsCluster.getNameNodePort() + "/"; - config.setHdfsuri(hdfsURI); - DistributedFileSystem fileSystem = hdfsCluster.getFileSystem(); + hdfsCluster = new TestMiniClusterFactory().create(config, baseDir); + fs = new TestFileSystemFactory().create(config.getHdfsuri()); - // ====== Init HDFS File System Object - Configuration fsConf = new Configuration(); - // Set FileSystem URI - fsConf.set("fs.defaultFS", hdfsURI); - // Because of Maven - fsConf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); - fsConf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); - // Set HADOOP user - System.setProperty("HADOOP_USER_NAME", "hdfs"); - System.setProperty("hadoop.home.dir", "/"); - fs = FileSystem.get(URI.create(hdfsURI), fsConf); // Inserts pre-made avro-files with old timestamps to HDFS, which are normally generated during data ingestion from mock kafka consumer. String path = config.getHdfsPath() + "/" + "testConsumerTopic"; // "hdfs:///opt/teragrep/cfe_39/srv/testConsumerTopic" diff --git a/src/test/java/com/teragrep/cfe_39/TestFileSystemFactory.java b/src/test/java/com/teragrep/cfe_39/TestFileSystemFactory.java new file mode 100644 index 00000000..af8ff781 --- /dev/null +++ b/src/test/java/com/teragrep/cfe_39/TestFileSystemFactory.java @@ -0,0 +1,71 @@ +/* + * HDFS Data Ingestion for PTH_06 use CFE-39 + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. + */ +package com.teragrep.cfe_39; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; + +import java.io.IOException; +import java.net.URI; + +public class TestFileSystemFactory { + + public FileSystem create(String hdfsURI) throws IOException { + FileSystem fs; + // ====== Init HDFS File System Object + Configuration fsConf = new Configuration(); + // Set FileSystem URI + fsConf.set("fs.defaultFS", hdfsURI); + // Because of Maven + fsConf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); + fsConf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); + // Set HADOOP user + System.setProperty("HADOOP_USER_NAME", "hdfs"); + System.setProperty("hadoop.home.dir", "/"); + fs = FileSystem.get(URI.create(hdfsURI), fsConf); + return fs; + } +} diff --git a/src/test/java/com/teragrep/cfe_39/TestMiniClusterFactory.java b/src/test/java/com/teragrep/cfe_39/TestMiniClusterFactory.java new file mode 100644 index 00000000..e7fbfb8f --- /dev/null +++ b/src/test/java/com/teragrep/cfe_39/TestMiniClusterFactory.java @@ -0,0 +1,70 @@ +/* + * HDFS Data Ingestion for PTH_06 use CFE-39 + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. + */ +package com.teragrep.cfe_39; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.apache.hadoop.hdfs.MiniDFSCluster; + +import java.io.File; +import java.io.IOException; + +// Helper class for creating FileSystem objects. +public class TestMiniClusterFactory { + + public MiniDFSCluster create(Config config, File baseDir) throws IOException { + MiniDFSCluster hdfsCluster; + // Create a HDFS miniCluster + Configuration conf = new Configuration(); + conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, baseDir.getAbsolutePath()); + MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf); + hdfsCluster = builder.build(); + String hdfsURI = "hdfs://localhost:" + hdfsCluster.getNameNodePort() + "/"; + config.setHdfsuri(hdfsURI); + DistributedFileSystem fileSystem = hdfsCluster.getFileSystem(); + return hdfsCluster; + } +} From 78d809e8e2e0fc6cd270f416987f8df7d9a0dffc Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Thu, 27 Jun 2024 13:04:53 +0300 Subject: [PATCH 121/146] Fixed logging. --- .../com/teragrep/cfe_39/consumers/kafka/HdfsDataIngestion.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HdfsDataIngestion.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HdfsDataIngestion.java index 0c60578f..460f9448 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HdfsDataIngestion.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HdfsDataIngestion.java @@ -256,7 +256,7 @@ private void topicScan(DurationStatistics durationStatistics, List durationStatistics.addAndGetThreads(1); } catch (SQLException sqlException) { - LOGGER.error("Topic <{}> not activated due to reader creation error: ", k, sqlException); + LOGGER.error("Topic <{}> not activated due to reader creation error: {}", k, sqlException); } }); durationStatistics.report(); From 5df2c15966623a094cda2172c067cc964dd10724 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Mon, 1 Jul 2024 11:19:58 +0300 Subject: [PATCH 122/146] Implemented Main.java, rpm packaging and GitHub workflows. --- .github/workflows/upload_release.yaml | 43 +++++++ pom.xml | 46 ++++++- rpm/rpm.pom.xml | 127 ++++++++++++++++++++ src/main/assembly/jar-with-dependencies.xml | 21 ++++ src/main/java/com/teragrep/cfe_39/Main.java | 75 ++++++++++++ 5 files changed, 309 insertions(+), 3 deletions(-) create mode 100644 .github/workflows/upload_release.yaml create mode 100644 rpm/rpm.pom.xml create mode 100644 src/main/assembly/jar-with-dependencies.xml create mode 100644 src/main/java/com/teragrep/cfe_39/Main.java diff --git a/.github/workflows/upload_release.yaml b/.github/workflows/upload_release.yaml new file mode 100644 index 00000000..6c14eeca --- /dev/null +++ b/.github/workflows/upload_release.yaml @@ -0,0 +1,43 @@ +name: Upload Release + +on: + release: + types: [published] + +jobs: + upload: + name: Upload + runs-on: ubuntu- + permissions: + contents: write + + steps: + - uses: actions/checkout@v3 + with: + fetch-depth: 0 + + - name: Set up JDK 8 + uses: actions/setup-java@v3 + with: + java-version: '8' + distribution: 'temurin' + server-id: github + settings-path: ${{ github.workspace }} + + + - name: Package jar + run: mvn --batch-mode -Drevision=${{ github.event.release.tag_name }} -Dsha1= -Dchangelist= clean package + env: + GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} + + - name: Package rpm + run: cd rpm/ && mvn --batch-mode -Drevision=${{ github.event.release.tag_name }} -Dsha1= -Dchangelist= -f rpm.pom.xml package + env: + GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} + + - name: Attach rpm to release + uses: softprops/action-gh-release@v1 + with: + files: | + rpm/target/rpm/com.teragrep-cfe_39/RPMS/noarch/com.teragrep-cfe_39-*.noarch.rpm + target/cfe_39-*-jar-with-dependencies.jar \ No newline at end of file diff --git a/pom.xml b/pom.xml index a66b3d0d..b2220cb8 100644 --- a/pom.xml +++ b/pom.xml @@ -146,6 +146,7 @@
+ ${project.artifactId}-${revision}${changelist}${sha1} org.apache.avro @@ -167,12 +168,51 @@ org.apache.maven.plugins maven-compiler-plugin - 3.8.1 + 3.12.1 - 1.8 - 1.8 + -Xlint:all + ${java.version} + ${java.version} + + org.apache.maven.plugins + maven-jar-plugin + 3.3.0 + + + + true + true + + + + + + org.apache.maven.plugins + maven-assembly-plugin + 3.6.0 + + + src/main/assembly/jar-with-dependencies.xml + + + + com.teragrep.cfe_39.Main + true + + + + + + make-assembly + + single + + package + + + org.apache.maven.plugins maven-enforcer-plugin diff --git a/rpm/rpm.pom.xml b/rpm/rpm.pom.xml new file mode 100644 index 00000000..2eefdddc --- /dev/null +++ b/rpm/rpm.pom.xml @@ -0,0 +1,127 @@ + + + rpm + 4.0.0 + cfe_39 + ${revision}${sha1}${changelist}.${env.TARGET_OS}.x86_64 + cfe_39 + cfe_39 + com.teragrep + + UTF-8 + 1.8 + 1.8 + 1.8 + 0.0.1 + -SNAPSHOT + + + + ${project.basedir}/target + + + maven-enforcer-plugin + 3.4.1 + + + enforce + none + + + enforce-maven + + enforce + + + + + 3.2.5 + + + + + + + + org.codehaus.mojo + rpm-maven-plugin + 2.2.0 + true + + + default-rpm + + rpm + + package + + + + ${project.groupId}-${project.artifactId} + ${project.groupId}-${project.artifactId} + ${project.version} + ${env.BUILD_ID} + Proprietary + teragrep Log Management Suite + https://teragrep.com/ + teragrep <servicedesk@teragrep.com> + teragrep/LogManagementSuite + false + root + root + 0644 + 0755 + + _build_id_links none + __provides_exclude ^osgi\\(.*$ + __requires_exclude ^osgi\\(.*$ + + + + /opt/teragrep/${project.artifactId}/lib + true + 755 + 755 + root + root + true + + + ${project.basedir}/../target/cfe_39-${revision}${sha1}${changelist}-jar-with-dependencies.jar + cfe_39.jar + + + + + + + + org.apache.maven.plugins + maven-deploy-plugin + 3.1.1 + + true + + + + org.apache.maven.plugins + maven-compiler-plugin + 3.12.1 + + + default-compile + compile + + compile + + + true + + + + + + + \ No newline at end of file diff --git a/src/main/assembly/jar-with-dependencies.xml b/src/main/assembly/jar-with-dependencies.xml new file mode 100644 index 00000000..ff9c3a29 --- /dev/null +++ b/src/main/assembly/jar-with-dependencies.xml @@ -0,0 +1,21 @@ + + jar-with-dependencies + + jar + + false + + + metaInf-services + + + + + true + true + runtime + + + \ No newline at end of file diff --git a/src/main/java/com/teragrep/cfe_39/Main.java b/src/main/java/com/teragrep/cfe_39/Main.java new file mode 100644 index 00000000..5be7a465 --- /dev/null +++ b/src/main/java/com/teragrep/cfe_39/Main.java @@ -0,0 +1,75 @@ +/* + * HDFS Data Ingestion for PTH_06 use CFE-39 + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. + */ +package com.teragrep.cfe_39; + +import com.teragrep.cfe_39.consumers.kafka.HdfsDataIngestion; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; + +public class Main { + + private static final Logger LOGGER = LoggerFactory.getLogger(Main.class); + + public static void main(String[] args) throws Exception { + Config config = null; + try { + config = new Config(); + } + catch (IOException e) { + System.out.println("Can't load config: " + e); + System.exit(1); + } + catch (IllegalArgumentException e) { + System.out.println("Got invalid config: " + e); + System.exit(1); + } + LOGGER.info("Running main program"); + HdfsDataIngestion hdfsDataIngestion = new HdfsDataIngestion(config); + hdfsDataIngestion.run(); + } +} From 825ba1ee88bfa1d0f4a0c653afd5d58619ac0959 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Mon, 1 Jul 2024 11:37:38 +0300 Subject: [PATCH 123/146] Fixed wrong json format usage in HdfsTest.java. --- src/test/java/com/teragrep/cfe_39/HdfsTest.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/test/java/com/teragrep/cfe_39/HdfsTest.java b/src/test/java/com/teragrep/cfe_39/HdfsTest.java index 423577e7..ca4a1a86 100644 --- a/src/test/java/com/teragrep/cfe_39/HdfsTest.java +++ b/src/test/java/com/teragrep/cfe_39/HdfsTest.java @@ -110,7 +110,7 @@ public void hdfsWriteTest() { Assertions.assertTrue(targetFile.toFile().exists()); File avroFile = new File(targetFile.toUri()); JsonObject recordOffsetJo = JsonParser - .parseString("{\"topic\":\"testConsumerTopic\", \"partition\":\"0\", \"offset\":\"9\"}") + .parseString("{\"topic\":\"testConsumerTopic\", \"partition\":0, \"offset\":9}") .getAsJsonObject(); try (HDFSWrite writer = new HDFSWrite(config, recordOffsetJo)) { writer.commit(avroFile); // commits avroFile to HDFS and deletes avroFile afterward. @@ -128,7 +128,7 @@ public void hdfsWriteTest() { Assertions.assertTrue(targetFile.toFile().exists()); avroFile = new File(config.getQueueDirectory() + "/0.13"); recordOffsetJo = JsonParser - .parseString("{\"topic\":\"testConsumerTopic\", \"partition\":\"0\", \"offset\":\"13\"}") + .parseString("{\"topic\":\"testConsumerTopic\", \"partition\":0, \"offset\":13}") .getAsJsonObject(); try (HDFSWrite writer = new HDFSWrite(config, recordOffsetJo)) { writer.commit(avroFile); // commits avroFile to HDFS and deletes avroFile afterward. @@ -158,7 +158,7 @@ public void hdfsWriteExceptionTest() { Assertions.assertTrue(targetFile.toFile().exists()); File avroFile = new File(targetFile.toUri()); JsonObject recordOffsetJo = JsonParser - .parseString("{\"topic\":\"testConsumerTopic\", \"partition\":\"0\", \"offset\":\"9\"}") + .parseString("{\"topic\":\"testConsumerTopic\", \"partition\":0, \"offset\":9}") .getAsJsonObject(); try (HDFSWrite writer = new HDFSWrite(config, recordOffsetJo)) { writer.commit(avroFile); // commits avroFile to HDFS and deletes avroFile afterward. @@ -172,7 +172,7 @@ public void hdfsWriteExceptionTest() { Assertions.assertTrue(targetFile.toFile().exists()); avroFile = new File(config.getQueueDirectory() + "/0.9"); recordOffsetJo = JsonParser - .parseString("{\"topic\":\"testConsumerTopic\", \"partition\":\"0\", \"offset\":\"9\"}") + .parseString("{\"topic\":\"testConsumerTopic\", \"partition\":0, \"offset\":9}") .getAsJsonObject(); HDFSWrite writer = new HDFSWrite(config, recordOffsetJo); File finalAvroFile = avroFile; From 9127524c8fcdc8e5db2472a1699999dc30fc630a Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Mon, 1 Jul 2024 11:49:05 +0300 Subject: [PATCH 124/146] Fixed logging. --- src/main/java/com/teragrep/cfe_39/Main.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/java/com/teragrep/cfe_39/Main.java b/src/main/java/com/teragrep/cfe_39/Main.java index 5be7a465..bb4e633c 100644 --- a/src/main/java/com/teragrep/cfe_39/Main.java +++ b/src/main/java/com/teragrep/cfe_39/Main.java @@ -61,11 +61,11 @@ public static void main(String[] args) throws Exception { config = new Config(); } catch (IOException e) { - System.out.println("Can't load config: " + e); + LOGGER.error("Can't load config: ", e); System.exit(1); } catch (IllegalArgumentException e) { - System.out.println("Got invalid config: " + e); + LOGGER.error("Got invalid config: ", e); System.exit(1); } LOGGER.info("Running main program"); From 51c7f4c286a86cc6bb7e4ce844b4638ba4873a03 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Mon, 1 Jul 2024 12:14:27 +0300 Subject: [PATCH 125/146] Removed unneeded System.setProperty. --- src/main/java/com/teragrep/cfe_39/Config.java | 1 - 1 file changed, 1 deletion(-) diff --git a/src/main/java/com/teragrep/cfe_39/Config.java b/src/main/java/com/teragrep/cfe_39/Config.java index 249828ac..305d67ec 100644 --- a/src/main/java/com/teragrep/cfe_39/Config.java +++ b/src/main/java/com/teragrep/cfe_39/Config.java @@ -124,7 +124,6 @@ public Config() throws IOException { if (!(new File(loginConfig)).isFile()) { throw new IOException("File '" + loginConfig + "' set by java.security.auth.login.config does not exist"); } - System.setProperty("java.security.auth.login.config", loginConfig); // Just for loggers to work Path log4j2Config = Paths From 06497539abcc630a07e71a2d28b19e1e0c2a1dc3 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Mon, 1 Jul 2024 12:18:39 +0300 Subject: [PATCH 126/146] Improved logging format. --- .../com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java index 197b5711..b82f5111 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java @@ -300,9 +300,8 @@ public void accept(List recordOffsetObjectList) { if (LOGGER.isDebugEnabled()) { LOGGER .debug( - "Target file size reached, file <{}> stored to <{}> in HDFS", syslogFile - .getName(), - lastObjectJo.get("topic").getAsString() + "/" + lastObjectJo.get("partition").getAsString() + "." + lastObjectJo.get("offset").getAsString() + "Target file size reached, file <{}> stored to <{}/{}.{}> in HDFS", + syslogFile.getName(), lastObjectJo.get("topic").getAsString(), lastObjectJo.get("partition").getAsString(), lastObjectJo.get("offset").getAsString() ); } } From f3601d3bc7dd66b01b2e84c4c31169e277829a8d Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Mon, 1 Jul 2024 12:20:01 +0300 Subject: [PATCH 127/146] Fixed error in json printer format. --- .../java/com/teragrep/cfe_39/consumers/kafka/NullOffset.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/NullOffset.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/NullOffset.java index 7f730df1..08fa3f22 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/NullOffset.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/NullOffset.java @@ -60,6 +60,6 @@ public byte[] getRecord() { @Override public String offsetToJSON() { - return "{\"topic\":\"Not available\", \"partition\":\"0\", \"offset\":\"0\"}"; + return "{\"topic\":\"Not available\", \"partition\":0, \"offset\":0}"; } } From f1d63448bc20f3539ac3d1d29f8d1204fecdbbd8 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Tue, 2 Jul 2024 12:10:03 +0300 Subject: [PATCH 128/146] Moved the pre-generated avro/hdfs files to test resources. --- src/test/java/com/teragrep/cfe_39/HdfsTest.java | 6 +++--- .../teragrep/cfe_39/Ingestion1Old1NewFileTest.java | 2 +- .../com/teragrep/cfe_39/Ingestion2NewFilesTest.java | 2 +- .../com/teragrep/cfe_39/Ingestion2OldFilesTest.java | 2 +- .../com/teragrep/cfe_39/PruningOneNewFileTest.java | 2 +- .../com/teragrep/cfe_39/PruningOneOldFileTest.java | 2 +- .../cfe_39/PruningOneOldOneNewFileTest.java | 2 +- .../com/teragrep/cfe_39/PruningTwoNewFilesTest.java | 2 +- .../com/teragrep/cfe_39/PruningTwoOldFilesTest.java | 2 +- .../cfe_39 => resources}/mockHdfsFiles/0.13 | Bin .../teragrep/cfe_39 => resources}/mockHdfsFiles/0.9 | Bin 11 files changed, 11 insertions(+), 11 deletions(-) rename src/test/{java/com/teragrep/cfe_39 => resources}/mockHdfsFiles/0.13 (100%) rename src/test/{java/com/teragrep/cfe_39 => resources}/mockHdfsFiles/0.9 (100%) diff --git a/src/test/java/com/teragrep/cfe_39/HdfsTest.java b/src/test/java/com/teragrep/cfe_39/HdfsTest.java index ca4a1a86..346fa25a 100644 --- a/src/test/java/com/teragrep/cfe_39/HdfsTest.java +++ b/src/test/java/com/teragrep/cfe_39/HdfsTest.java @@ -101,7 +101,7 @@ public void hdfsWriteTest() { Assertions.assertFalse(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic"))); // writer.commit will delete the file that is given as an input argument. Copy the mock files to another directory so the deletion can be asserted properly too. - String pathname = System.getProperty("user.dir") + "/src/test/java/com/teragrep/cfe_39/mockHdfsFiles/0.9"; + String pathname = System.getProperty("user.dir") + "/src/test/resources/mockHdfsFiles/0.9"; java.nio.file.Path sourceFile = Paths.get(pathname); java.nio.file.Path targetDir = Paths.get(config.getQueueDirectory()); java.nio.file.Path targetFile = targetDir.resolve(sourceFile.getFileName()); @@ -120,7 +120,7 @@ public void hdfsWriteTest() { .assertEquals(1, fs.listStatus(new Path(config.getHdfsPath() + "/" + "testConsumerTopic")).length); Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic" + "/" + "0.9"))); - pathname = System.getProperty("user.dir") + "/src/test/java/com/teragrep/cfe_39/mockHdfsFiles/0.13"; + pathname = System.getProperty("user.dir") + "/src/test/resources/mockHdfsFiles/0.13"; sourceFile = Paths.get(pathname); targetDir = Paths.get(config.getQueueDirectory()); targetFile = targetDir.resolve(sourceFile.getFileName()); @@ -148,7 +148,7 @@ public void hdfsWriteExceptionTest() { Assertions.assertFalse(fs.exists(new Path(config.getHdfsPath() + "/" + "testConsumerTopic"))); // writer.commit will delete the source file that is given as an input argument. Copy the mock file to another directory so the deletion of the source file can be asserted properly. - String pathname = System.getProperty("user.dir") + "/src/test/java/com/teragrep/cfe_39/mockHdfsFiles/0.9"; + String pathname = System.getProperty("user.dir") + "/src/test/resources/mockHdfsFiles/0.9"; java.nio.file.Path sourceFile = Paths.get(pathname); java.nio.file.Path targetDir = Paths.get(config.getQueueDirectory()); java.nio.file.Path targetFile = targetDir.resolve(sourceFile.getFileName()); diff --git a/src/test/java/com/teragrep/cfe_39/Ingestion1Old1NewFileTest.java b/src/test/java/com/teragrep/cfe_39/Ingestion1Old1NewFileTest.java index 548d490c..c9fd7fb2 100644 --- a/src/test/java/com/teragrep/cfe_39/Ingestion1Old1NewFileTest.java +++ b/src/test/java/com/teragrep/cfe_39/Ingestion1Old1NewFileTest.java @@ -91,7 +91,7 @@ public void startMiniCluster() { // Create new Directory fs.mkdirs(newDirectoryPath); LOGGER.debug("Path {} created.", path); - String dir = System.getProperty("user.dir") + "/src/test/java/com/teragrep/cfe_39/mockHdfsFiles"; + String dir = System.getProperty("user.dir") + "/src/test/resources/mockHdfsFiles"; Set listOfFiles = Stream .of(Objects.requireNonNull(new File(dir).listFiles())) .filter(file -> !file.isDirectory()) diff --git a/src/test/java/com/teragrep/cfe_39/Ingestion2NewFilesTest.java b/src/test/java/com/teragrep/cfe_39/Ingestion2NewFilesTest.java index 7479f1aa..75a54456 100644 --- a/src/test/java/com/teragrep/cfe_39/Ingestion2NewFilesTest.java +++ b/src/test/java/com/teragrep/cfe_39/Ingestion2NewFilesTest.java @@ -96,7 +96,7 @@ public void startMiniCluster() { // Create new Directory fs.mkdirs(newDirectoryPath); LOGGER.debug("Path {} created.", path); - String dir = System.getProperty("user.dir") + "/src/test/java/com/teragrep/cfe_39/mockHdfsFiles"; + String dir = System.getProperty("user.dir") + "/src/test/resources/mockHdfsFiles"; Set listOfFiles = Stream .of(Objects.requireNonNull(new File(dir).listFiles())) .filter(file -> !file.isDirectory()) diff --git a/src/test/java/com/teragrep/cfe_39/Ingestion2OldFilesTest.java b/src/test/java/com/teragrep/cfe_39/Ingestion2OldFilesTest.java index d40895bd..50812403 100644 --- a/src/test/java/com/teragrep/cfe_39/Ingestion2OldFilesTest.java +++ b/src/test/java/com/teragrep/cfe_39/Ingestion2OldFilesTest.java @@ -92,7 +92,7 @@ public void startMiniCluster() { fs.mkdirs(newDirectoryPath); LOGGER.debug("Path {} created.", path); - String dir = System.getProperty("user.dir") + "/src/test/java/com/teragrep/cfe_39/mockHdfsFiles"; + String dir = System.getProperty("user.dir") + "/src/test/resources/mockHdfsFiles"; Set listOfFiles = Stream .of(Objects.requireNonNull(new File(dir).listFiles())) .filter(file -> !file.isDirectory()) diff --git a/src/test/java/com/teragrep/cfe_39/PruningOneNewFileTest.java b/src/test/java/com/teragrep/cfe_39/PruningOneNewFileTest.java index 7b210586..4fd750db 100644 --- a/src/test/java/com/teragrep/cfe_39/PruningOneNewFileTest.java +++ b/src/test/java/com/teragrep/cfe_39/PruningOneNewFileTest.java @@ -91,7 +91,7 @@ public void startMiniCluster() { // Create new Directory fs.mkdirs(newDirectoryPath); LOGGER.debug("Path {} created.", path); - String dir = System.getProperty("user.dir") + "/src/test/java/com/teragrep/cfe_39/mockHdfsFiles"; + String dir = System.getProperty("user.dir") + "/src/test/resources/mockHdfsFiles"; Set listOfFiles = Stream .of(Objects.requireNonNull(new File(dir).listFiles())) .filter(file -> !file.isDirectory()) diff --git a/src/test/java/com/teragrep/cfe_39/PruningOneOldFileTest.java b/src/test/java/com/teragrep/cfe_39/PruningOneOldFileTest.java index 1a5c4021..155eaca2 100644 --- a/src/test/java/com/teragrep/cfe_39/PruningOneOldFileTest.java +++ b/src/test/java/com/teragrep/cfe_39/PruningOneOldFileTest.java @@ -91,7 +91,7 @@ public void startMiniCluster() { // Create new Directory fs.mkdirs(newDirectoryPath); LOGGER.debug("Path {} created.", path); - String dir = System.getProperty("user.dir") + "/src/test/java/com/teragrep/cfe_39/mockHdfsFiles"; + String dir = System.getProperty("user.dir") + "/src/test/resources/mockHdfsFiles"; Set listOfFiles = Stream .of(Objects.requireNonNull(new File(dir).listFiles())) .filter(file -> !file.isDirectory()) diff --git a/src/test/java/com/teragrep/cfe_39/PruningOneOldOneNewFileTest.java b/src/test/java/com/teragrep/cfe_39/PruningOneOldOneNewFileTest.java index 6bf1982f..0a988618 100644 --- a/src/test/java/com/teragrep/cfe_39/PruningOneOldOneNewFileTest.java +++ b/src/test/java/com/teragrep/cfe_39/PruningOneOldOneNewFileTest.java @@ -92,7 +92,7 @@ public void startMiniCluster() { // Create new Directory fs.mkdirs(newDirectoryPath); LOGGER.debug("Path {} created.", path); - String dir = System.getProperty("user.dir") + "/src/test/java/com/teragrep/cfe_39/mockHdfsFiles"; + String dir = System.getProperty("user.dir") + "/src/test/resources/mockHdfsFiles"; Set listOfFiles = Stream .of(Objects.requireNonNull(new File(dir).listFiles())) .filter(file -> !file.isDirectory()) diff --git a/src/test/java/com/teragrep/cfe_39/PruningTwoNewFilesTest.java b/src/test/java/com/teragrep/cfe_39/PruningTwoNewFilesTest.java index c79d4377..fb381d59 100644 --- a/src/test/java/com/teragrep/cfe_39/PruningTwoNewFilesTest.java +++ b/src/test/java/com/teragrep/cfe_39/PruningTwoNewFilesTest.java @@ -93,7 +93,7 @@ public void startMiniCluster() { fs.mkdirs(newDirectoryPath); LOGGER.debug("Path {} created.", path); - String dir = System.getProperty("user.dir") + "/src/test/java/com/teragrep/cfe_39/mockHdfsFiles"; + String dir = System.getProperty("user.dir") + "/src/test/resources/mockHdfsFiles"; Set listOfFiles = Stream .of(Objects.requireNonNull(new File(dir).listFiles())) .filter(file -> !file.isDirectory()) diff --git a/src/test/java/com/teragrep/cfe_39/PruningTwoOldFilesTest.java b/src/test/java/com/teragrep/cfe_39/PruningTwoOldFilesTest.java index 2c570500..1f2ed5d2 100644 --- a/src/test/java/com/teragrep/cfe_39/PruningTwoOldFilesTest.java +++ b/src/test/java/com/teragrep/cfe_39/PruningTwoOldFilesTest.java @@ -92,7 +92,7 @@ public void startMiniCluster() { // Create new Directory fs.mkdirs(newDirectoryPath); LOGGER.debug("Path {} created.", path); - String dir = System.getProperty("user.dir") + "/src/test/java/com/teragrep/cfe_39/mockHdfsFiles"; + String dir = System.getProperty("user.dir") + "/src/test/resources/mockHdfsFiles"; Set listOfFiles = Stream .of(Objects.requireNonNull(new File(dir).listFiles())) .filter(file -> !file.isDirectory()) diff --git a/src/test/java/com/teragrep/cfe_39/mockHdfsFiles/0.13 b/src/test/resources/mockHdfsFiles/0.13 similarity index 100% rename from src/test/java/com/teragrep/cfe_39/mockHdfsFiles/0.13 rename to src/test/resources/mockHdfsFiles/0.13 diff --git a/src/test/java/com/teragrep/cfe_39/mockHdfsFiles/0.9 b/src/test/resources/mockHdfsFiles/0.9 similarity index 100% rename from src/test/java/com/teragrep/cfe_39/mockHdfsFiles/0.9 rename to src/test/resources/mockHdfsFiles/0.9 From 3183e970b5af3455425a153ee24fad6f64f381de Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Tue, 2 Jul 2024 16:26:16 +0300 Subject: [PATCH 129/146] Implemented configuration tests. Implemented configuration checks. Added test configuration files and their usage. Fixes to Config.java resource locations. --- rpm/resources/application.properties | 13 ++++- src/main/java/com/teragrep/cfe_39/Config.java | 49 ++++++++++++++----- .../java/com/teragrep/cfe_39/ConfigTest.java | 19 +++++-- .../java/com/teragrep/cfe_39/HdfsTest.java | 3 ++ .../teragrep/cfe_39/Ingestion0FilesTest.java | 3 ++ .../cfe_39/Ingestion1Old1NewFileTest.java | 3 ++ .../cfe_39/Ingestion2NewFilesTest.java | 3 ++ .../cfe_39/Ingestion2OldFilesTest.java | 3 ++ .../teragrep/cfe_39/KafkaConsumerTest.java | 3 ++ .../teragrep/cfe_39/PruningNoFilesTest.java | 3 ++ .../cfe_39/PruningOneNewFileTest.java | 3 ++ .../cfe_39/PruningOneOldFileTest.java | 3 ++ .../cfe_39/PruningOneOldOneNewFileTest.java | 3 ++ .../cfe_39/PruningTwoNewFilesTest.java | 3 ++ .../cfe_39/PruningTwoOldFilesTest.java | 3 ++ .../resources/broken.application.properties | 37 ++++++++++++++ .../resources/valid.application.properties | 39 +++++++++++++++ 17 files changed, 176 insertions(+), 17 deletions(-) create mode 100644 src/test/resources/broken.application.properties create mode 100644 src/test/resources/valid.application.properties diff --git a/rpm/resources/application.properties b/rpm/resources/application.properties index e1f34301..49cc6e61 100644 --- a/rpm/resources/application.properties +++ b/rpm/resources/application.properties @@ -31,4 +31,15 @@ queueDirectory=/home/user/IdeaProjects/cfe_39/etc/AVRO/ # The maximum file size for AVRO-files that are to be stored in HDFS database. maximumFileSize=3000 # HDFS pruning, use 157784760000 value while testing HDFS writes to ensure the test records are not pruned. 157784760000L -pruneOffset=157784760000 \ No newline at end of file +pruneOffset=157784760000 +# HDFS uri +hdfsuri=hdfs://localhost:45937/ +# Kerberos +java.security.krb5.kdc=test +java.security.krb5.realm=test +hadoop.security.authentication=test +hadoop.security.authorization=test +dfs.namenode.kerberos.principal.pattern=test +KerberosKeytabUser=test +KerberosKeytabPath=test +dfs.client.use.datanode.hostname=false \ No newline at end of file diff --git a/src/main/java/com/teragrep/cfe_39/Config.java b/src/main/java/com/teragrep/cfe_39/Config.java index 305d67ec..5e95cfd7 100644 --- a/src/main/java/com/teragrep/cfe_39/Config.java +++ b/src/main/java/com/teragrep/cfe_39/Config.java @@ -78,12 +78,10 @@ public class Config { private final int numOfConsumers; private final long pruneOffset; - // TODO: Set up configuration check for important parameters. - public Config() throws IOException { Properties properties = new Properties(); Path configPath = Paths - .get(System.getProperty("cfe_39.config.location", System.getProperty("user.dir") + "/etc/application.properties")); + .get(System.getProperty("cfe_39.config.location", System.getProperty("user.dir") + "/rpm/resources/application.properties")); LOGGER.info("Loading application config <[{}]>", configPath.toAbsolutePath()); try (InputStream inputStream = Files.newInputStream(configPath)) { @@ -93,7 +91,10 @@ public Config() throws IOException { // HDFS this.hdfsPath = properties.getProperty("hdfsPath", "hdfs:///opt/teragrep/cfe_39/srv/"); - this.hdfsuri = properties.getProperty("hdfsuri", "hdfs://localhost:45937/"); + this.hdfsuri = properties.getProperty("hdfsuri"); + if (this.hdfsuri == null) { + throw new IllegalArgumentException("hdfsuri not set"); + } // HDFS pruning this.pruneOffset = Long.parseLong(properties.getProperty("pruneOffset", "172800000")); @@ -103,13 +104,34 @@ public Config() throws IOException { this.maximumFileSize = Long.parseLong(properties.getProperty("maximumFileSize", "60800000")); // kerberos - this.kerberosHost = properties.getProperty("java.security.krb5.kdc", ""); - this.kerberosRealm = properties.getProperty("java.security.krb5.realm", ""); - this.hadoopAuthentication = properties.getProperty("hadoop.security.authentication", ""); - this.hadoopAuthorization = properties.getProperty("hadoop.security.authorization", ""); - this.kerberosPrincipal = properties.getProperty("dfs.namenode.kerberos.principal.pattern", ""); - this.kerberosKeytabUser = properties.getProperty("KerberosKeytabUser", ""); - this.kerberosKeytabPath = properties.getProperty("KerberosKeytabPath", ""); + this.kerberosHost = properties.getProperty("java.security.krb5.kdc"); + if (this.kerberosHost == null) { + throw new IllegalArgumentException("kerberosHost not set"); + } + this.kerberosRealm = properties.getProperty("java.security.krb5.realm"); + if (this.kerberosRealm == null) { + throw new IllegalArgumentException("kerberosRealm not set"); + } + this.hadoopAuthentication = properties.getProperty("hadoop.security.authentication"); + if (this.hadoopAuthentication == null) { + throw new IllegalArgumentException("hadoopAuthentication not set"); + } + this.hadoopAuthorization = properties.getProperty("hadoop.security.authorization"); + if (this.hadoopAuthorization == null) { + throw new IllegalArgumentException("hadoopAuthorization not set"); + } + this.kerberosPrincipal = properties.getProperty("dfs.namenode.kerberos.principal.pattern"); + if (this.kerberosPrincipal == null) { + throw new IllegalArgumentException("kerberosPrincipal not set"); + } + this.kerberosKeytabUser = properties.getProperty("KerberosKeytabUser"); + if (this.kerberosKeytabUser == null) { + throw new IllegalArgumentException("kerberosKeytabUser not set"); + } + this.kerberosKeytabPath = properties.getProperty("KerberosKeytabPath"); + if (this.kerberosKeytabPath == null) { + throw new IllegalArgumentException("kerberosKeytabPath not set"); + } this.kerberosTestMode = properties.getProperty("dfs.client.use.datanode.hostname", "false"); // kafka @@ -117,7 +139,8 @@ public Config() throws IOException { this.numOfConsumers = Integer.parseInt(properties.getProperty("numOfConsumers", "1")); this.kafkaConsumerProperties = loadSubProperties(properties, "consumer."); - String loginConfig = properties.getProperty("java.security.auth.login.config"); + String loginConfig = properties + .getProperty("java.security.auth.login.config", System.getProperty("user.dir") + "/rpm/resources/config.jaas"); if (loginConfig == null) { throw new IOException("Property java.security.auth.login.config does not exist"); } @@ -127,7 +150,7 @@ public Config() throws IOException { // Just for loggers to work Path log4j2Config = Paths - .get(properties.getProperty("log4j2.configurationFile", System.getProperty("user.dir") + "/etc/log4j2.properties")); + .get(properties.getProperty("log4j2.configurationFile", System.getProperty("user.dir") + "/rpm/resources/log4j2.properties")); LOGGER.info("Loading log4j2 config from <[{}]>", log4j2Config.toRealPath()); Configurator.reconfigure(log4j2Config.toUri()); } diff --git a/src/test/java/com/teragrep/cfe_39/ConfigTest.java b/src/test/java/com/teragrep/cfe_39/ConfigTest.java index b8c41dc3..7fc13bf0 100644 --- a/src/test/java/com/teragrep/cfe_39/ConfigTest.java +++ b/src/test/java/com/teragrep/cfe_39/ConfigTest.java @@ -46,7 +46,6 @@ package com.teragrep.cfe_39; import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -59,10 +58,12 @@ public class ConfigTest { private static final Logger LOGGER = LoggerFactory.getLogger(ConfigTest.class); - @Disabled(value = "This code is WIP, leaving the test implementation to deployment step.") @Test - public void configTest() { + public void validConfigTest() { assertDoesNotThrow(() -> { + // Set system properties to use the valid configuration. + System + .setProperty("cfe_39.config.location", System.getProperty("user.dir") + "/src/test/resources/valid.application.properties"); Config config = new Config(); Properties readerKafkaProperties = config.getKafkaConsumerProperties(); // Test extracting useMockKafkaConsumer value from config. @@ -72,4 +73,16 @@ public void configTest() { LOGGER.debug("useMockKafkaConsumer: {}", useMockKafkaConsumer); }); } + + @Test + public void brokenConfigTest() { + // Set system properties to use the broken configuration. + System + .setProperty("cfe_39.config.location", System.getProperty("user.dir") + "/src/test/resources/broken.application.properties"); + // Test if the broken configuration throws the expected exception. + Exception e = Assertions.assertThrows(Exception.class, () -> { + Config config = new Config(); + }); + Assertions.assertEquals("hdfsuri not set", e.getMessage()); + } } diff --git a/src/test/java/com/teragrep/cfe_39/HdfsTest.java b/src/test/java/com/teragrep/cfe_39/HdfsTest.java index 346fa25a..315e6dac 100644 --- a/src/test/java/com/teragrep/cfe_39/HdfsTest.java +++ b/src/test/java/com/teragrep/cfe_39/HdfsTest.java @@ -76,6 +76,9 @@ public class HdfsTest { @BeforeEach public void startMiniCluster() { assertDoesNotThrow(() -> { + // Set system properties to use the valid configuration. + System + .setProperty("cfe_39.config.location", System.getProperty("user.dir") + "/src/test/resources/valid.application.properties"); config = new Config(); // Create a HDFS miniCluster baseDir = Files.createTempDirectory("test_hdfs").toFile().getAbsoluteFile(); diff --git a/src/test/java/com/teragrep/cfe_39/Ingestion0FilesTest.java b/src/test/java/com/teragrep/cfe_39/Ingestion0FilesTest.java index 648af468..23c52a85 100644 --- a/src/test/java/com/teragrep/cfe_39/Ingestion0FilesTest.java +++ b/src/test/java/com/teragrep/cfe_39/Ingestion0FilesTest.java @@ -76,6 +76,9 @@ public class Ingestion0FilesTest { @BeforeEach public void startMiniCluster() { assertDoesNotThrow(() -> { + // Set system properties to use the valid configuration. + System + .setProperty("cfe_39.config.location", System.getProperty("user.dir") + "/src/test/resources/valid.application.properties"); config = new Config(); // Create a HDFS miniCluster baseDir = Files.createTempDirectory("test_hdfs").toFile().getAbsoluteFile(); diff --git a/src/test/java/com/teragrep/cfe_39/Ingestion1Old1NewFileTest.java b/src/test/java/com/teragrep/cfe_39/Ingestion1Old1NewFileTest.java index c9fd7fb2..1ae2b380 100644 --- a/src/test/java/com/teragrep/cfe_39/Ingestion1Old1NewFileTest.java +++ b/src/test/java/com/teragrep/cfe_39/Ingestion1Old1NewFileTest.java @@ -78,6 +78,9 @@ public class Ingestion1Old1NewFileTest { @BeforeEach public void startMiniCluster() { assertDoesNotThrow(() -> { + // Set system properties to use the valid configuration. + System + .setProperty("cfe_39.config.location", System.getProperty("user.dir") + "/src/test/resources/valid.application.properties"); config = new Config(); // Create a HDFS miniCluster baseDir = Files.createTempDirectory("test_hdfs").toFile().getAbsoluteFile(); diff --git a/src/test/java/com/teragrep/cfe_39/Ingestion2NewFilesTest.java b/src/test/java/com/teragrep/cfe_39/Ingestion2NewFilesTest.java index 75a54456..7eaca4f9 100644 --- a/src/test/java/com/teragrep/cfe_39/Ingestion2NewFilesTest.java +++ b/src/test/java/com/teragrep/cfe_39/Ingestion2NewFilesTest.java @@ -83,6 +83,9 @@ public class Ingestion2NewFilesTest { @BeforeEach public void startMiniCluster() { assertDoesNotThrow(() -> { + // Set system properties to use the valid configuration. + System + .setProperty("cfe_39.config.location", System.getProperty("user.dir") + "/src/test/resources/valid.application.properties"); config = new Config(); // Create a HDFS miniCluster baseDir = Files.createTempDirectory("test_hdfs").toFile().getAbsoluteFile(); diff --git a/src/test/java/com/teragrep/cfe_39/Ingestion2OldFilesTest.java b/src/test/java/com/teragrep/cfe_39/Ingestion2OldFilesTest.java index 50812403..b9e110b4 100644 --- a/src/test/java/com/teragrep/cfe_39/Ingestion2OldFilesTest.java +++ b/src/test/java/com/teragrep/cfe_39/Ingestion2OldFilesTest.java @@ -78,6 +78,9 @@ public class Ingestion2OldFilesTest { @BeforeEach public void startMiniCluster() { assertDoesNotThrow(() -> { + // Set system properties to use the valid configuration. + System + .setProperty("cfe_39.config.location", System.getProperty("user.dir") + "/src/test/resources/valid.application.properties"); config = new Config(); // Create a HDFS miniCluster baseDir = Files.createTempDirectory("test_hdfs").toFile().getAbsoluteFile(); diff --git a/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java b/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java index 43d5e36b..593794be 100644 --- a/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java +++ b/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java @@ -67,6 +67,9 @@ public class KafkaConsumerTest { @Test public void readCoordinatorTest2Threads() { assertDoesNotThrow(() -> { + // Set system properties to use the valid configuration. + System + .setProperty("cfe_39.config.location", System.getProperty("user.dir") + "/src/test/resources/valid.application.properties"); Config config = new Config(); Map hdfsStartOffsets = new HashMap<>(); ArrayList> messages = new ArrayList<>(); diff --git a/src/test/java/com/teragrep/cfe_39/PruningNoFilesTest.java b/src/test/java/com/teragrep/cfe_39/PruningNoFilesTest.java index e04b4915..f89603d3 100644 --- a/src/test/java/com/teragrep/cfe_39/PruningNoFilesTest.java +++ b/src/test/java/com/teragrep/cfe_39/PruningNoFilesTest.java @@ -74,6 +74,9 @@ public class PruningNoFilesTest { @BeforeEach public void startMiniCluster() { assertDoesNotThrow(() -> { + // Set system properties to use the valid configuration. + System + .setProperty("cfe_39.config.location", System.getProperty("user.dir") + "/src/test/resources/valid.application.properties"); config = new Config(); // Create a HDFS miniCluster baseDir = Files.createTempDirectory("test_hdfs").toFile().getAbsoluteFile(); diff --git a/src/test/java/com/teragrep/cfe_39/PruningOneNewFileTest.java b/src/test/java/com/teragrep/cfe_39/PruningOneNewFileTest.java index 4fd750db..bcd06660 100644 --- a/src/test/java/com/teragrep/cfe_39/PruningOneNewFileTest.java +++ b/src/test/java/com/teragrep/cfe_39/PruningOneNewFileTest.java @@ -78,6 +78,9 @@ public class PruningOneNewFileTest { @BeforeEach public void startMiniCluster() { assertDoesNotThrow(() -> { + // Set system properties to use the valid configuration. + System + .setProperty("cfe_39.config.location", System.getProperty("user.dir") + "/src/test/resources/valid.application.properties"); config = new Config(); // Create a HDFS miniCluster baseDir = Files.createTempDirectory("test_hdfs").toFile().getAbsoluteFile(); diff --git a/src/test/java/com/teragrep/cfe_39/PruningOneOldFileTest.java b/src/test/java/com/teragrep/cfe_39/PruningOneOldFileTest.java index 155eaca2..0e7445f3 100644 --- a/src/test/java/com/teragrep/cfe_39/PruningOneOldFileTest.java +++ b/src/test/java/com/teragrep/cfe_39/PruningOneOldFileTest.java @@ -78,6 +78,9 @@ public class PruningOneOldFileTest { @BeforeEach public void startMiniCluster() { assertDoesNotThrow(() -> { + // Set system properties to use the valid configuration. + System + .setProperty("cfe_39.config.location", System.getProperty("user.dir") + "/src/test/resources/valid.application.properties"); config = new Config(); // Create a HDFS miniCluster baseDir = Files.createTempDirectory("test_hdfs").toFile().getAbsoluteFile(); diff --git a/src/test/java/com/teragrep/cfe_39/PruningOneOldOneNewFileTest.java b/src/test/java/com/teragrep/cfe_39/PruningOneOldOneNewFileTest.java index 0a988618..483e36dc 100644 --- a/src/test/java/com/teragrep/cfe_39/PruningOneOldOneNewFileTest.java +++ b/src/test/java/com/teragrep/cfe_39/PruningOneOldOneNewFileTest.java @@ -78,6 +78,9 @@ public class PruningOneOldOneNewFileTest { @BeforeEach public void startMiniCluster() { assertDoesNotThrow(() -> { + // Set system properties to use the valid configuration. + System + .setProperty("cfe_39.config.location", System.getProperty("user.dir") + "/src/test/resources/valid.application.properties"); config = new Config(); // Create a HDFS miniCluster baseDir = Files.createTempDirectory("test_hdfs").toFile().getAbsoluteFile(); diff --git a/src/test/java/com/teragrep/cfe_39/PruningTwoNewFilesTest.java b/src/test/java/com/teragrep/cfe_39/PruningTwoNewFilesTest.java index fb381d59..0f3b450c 100644 --- a/src/test/java/com/teragrep/cfe_39/PruningTwoNewFilesTest.java +++ b/src/test/java/com/teragrep/cfe_39/PruningTwoNewFilesTest.java @@ -78,6 +78,9 @@ public class PruningTwoNewFilesTest { @BeforeEach public void startMiniCluster() { assertDoesNotThrow(() -> { + // Set system properties to use the valid configuration. + System + .setProperty("cfe_39.config.location", System.getProperty("user.dir") + "/src/test/resources/valid.application.properties"); config = new Config(); // Create a HDFS miniCluster baseDir = Files.createTempDirectory("test_hdfs").toFile().getAbsoluteFile(); diff --git a/src/test/java/com/teragrep/cfe_39/PruningTwoOldFilesTest.java b/src/test/java/com/teragrep/cfe_39/PruningTwoOldFilesTest.java index 1f2ed5d2..0a5ae764 100644 --- a/src/test/java/com/teragrep/cfe_39/PruningTwoOldFilesTest.java +++ b/src/test/java/com/teragrep/cfe_39/PruningTwoOldFilesTest.java @@ -78,6 +78,9 @@ public class PruningTwoOldFilesTest { @BeforeEach public void startMiniCluster() { assertDoesNotThrow(() -> { + // Set system properties to use the valid configuration. + System + .setProperty("cfe_39.config.location", System.getProperty("user.dir") + "/src/test/resources/valid.application.properties"); config = new Config(); // Create a HDFS miniCluster baseDir = Files.createTempDirectory("test_hdfs").toFile().getAbsoluteFile(); diff --git a/src/test/resources/broken.application.properties b/src/test/resources/broken.application.properties new file mode 100644 index 00000000..2a6d929b --- /dev/null +++ b/src/test/resources/broken.application.properties @@ -0,0 +1,37 @@ +# What topics are searched from kafka, regex +queueTopicPattern=^testConsumerTopic-*$ +# Number of consumers created to the consumer groups +numOfConsumers=2 +# Kafka bootstrap servers +consumer.bootstrap.servers=test +# Offset, should not be touched +consumer.auto.offset.reset=earliest +# Autocommit, should not be touched +consumer.enable.auto.commit=false +# Consumer group id, this is to track the progress of reading hte topic +consumer.group.id=cfe_39 +# Used security protocol and mechanism +consumer.security.protocol=SASL_PLAINTEXT +consumer.sasl.mechanism=PLAIN +# Maximum records per batch, note that too big number will cause massive load and can cause timeouts to trigger +consumer.max.poll.records=500 +# How much data can be fetched in one go +consumer.fetch.max.bytes=1073741820 +# How long for request before timing out. Note that too big max poll records size can cause this to trigger +consumer.request.timeout.ms=300000 +consumer.max.poll.interval.ms=300000 +# For testing only, remove for prod. +consumer.useMockKafkaConsumer=true +# The maximum file size for AVRO-files that are to be stored in HDFS database. +maximumFileSize=3000 +# HDFS pruning, use 157784760000 value while testing HDFS writes to ensure the test records are not pruned. 157784760000L +pruneOffset=157784760000 +# Kerberos +java.security.krb5.kdc=test +java.security.krb5.realm=test +hadoop.security.authentication=test +hadoop.security.authorization=test +dfs.namenode.kerberos.principal.pattern=test +KerberosKeytabUser=test +KerberosKeytabPath=test +dfs.client.use.datanode.hostname=false \ No newline at end of file diff --git a/src/test/resources/valid.application.properties b/src/test/resources/valid.application.properties new file mode 100644 index 00000000..2f03cba0 --- /dev/null +++ b/src/test/resources/valid.application.properties @@ -0,0 +1,39 @@ +# What topics are searched from kafka, regex +queueTopicPattern=^testConsumerTopic-*$ +# Number of consumers created to the consumer groups +numOfConsumers=2 +# Kafka bootstrap servers +consumer.bootstrap.servers=test +# Offset, should not be touched +consumer.auto.offset.reset=earliest +# Autocommit, should not be touched +consumer.enable.auto.commit=false +# Consumer group id, this is to track the progress of reading hte topic +consumer.group.id=cfe_39 +# Used security protocol and mechanism +consumer.security.protocol=SASL_PLAINTEXT +consumer.sasl.mechanism=PLAIN +# Maximum records per batch, note that too big number will cause massive load and can cause timeouts to trigger +consumer.max.poll.records=500 +# How much data can be fetched in one go +consumer.fetch.max.bytes=1073741820 +# How long for request before timing out. Note that too big max poll records size can cause this to trigger +consumer.request.timeout.ms=300000 +consumer.max.poll.interval.ms=300000 +# For testing only, remove for prod. +consumer.useMockKafkaConsumer=true +# The maximum file size for AVRO-files that are to be stored in HDFS database. +maximumFileSize=3000 +# HDFS pruning, use 157784760000 value while testing HDFS writes to ensure the test records are not pruned. 157784760000L +pruneOffset=157784760000 +# HDFS uri +hdfsuri=hdfs://localhost:45937/ +# Kerberos +java.security.krb5.kdc=test +java.security.krb5.realm=test +hadoop.security.authentication=test +hadoop.security.authorization=test +dfs.namenode.kerberos.principal.pattern=test +KerberosKeytabUser=test +KerberosKeytabPath=test +dfs.client.use.datanode.hostname=false \ No newline at end of file From 42337f2feea82a76c8d51ca1a8955212f5243d0f Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Wed, 3 Jul 2024 11:00:07 +0300 Subject: [PATCH 130/146] Added more configuration checks. Changed default directory of application.properties. Updated example configuration file. --- rpm/resources/application.properties | 6 +++--- src/main/java/com/teragrep/cfe_39/Config.java | 8 +++++++- 2 files changed, 10 insertions(+), 4 deletions(-) diff --git a/rpm/resources/application.properties b/rpm/resources/application.properties index 49cc6e61..cdeff44e 100644 --- a/rpm/resources/application.properties +++ b/rpm/resources/application.properties @@ -1,7 +1,7 @@ # Kafka security configuration file -java.security.auth.login.config=/home/user/IdeaProjects/cfe_39/etc/config.jaas +java.security.auth.login.config=/opt/teragrep/cfe_39/etc/config.jaas # Logger settings -log4j2.configurationFile=/home/user/IdeaProjects/cfe_39/etc/log4j2.properties +log4j2.configurationFile=/opt/teragrep/cfe_39/etc/log4j2.properties # What topics are searched from kafka, regex queueTopicPattern=^testConsumerTopic-*$ # Number of consumers created to the consumer groups @@ -27,7 +27,7 @@ consumer.max.poll.interval.ms=300000 # For testing only, remove for prod. consumer.useMockKafkaConsumer=true # AVRO -queueDirectory=/home/user/IdeaProjects/cfe_39/etc/AVRO/ +queueDirectory=/opt/teragrep/cfe_39/etc/AVRO/ # The maximum file size for AVRO-files that are to be stored in HDFS database. maximumFileSize=3000 # HDFS pruning, use 157784760000 value while testing HDFS writes to ensure the test records are not pruned. 157784760000L diff --git a/src/main/java/com/teragrep/cfe_39/Config.java b/src/main/java/com/teragrep/cfe_39/Config.java index 5e95cfd7..65dad468 100644 --- a/src/main/java/com/teragrep/cfe_39/Config.java +++ b/src/main/java/com/teragrep/cfe_39/Config.java @@ -81,7 +81,7 @@ public class Config { public Config() throws IOException { Properties properties = new Properties(); Path configPath = Paths - .get(System.getProperty("cfe_39.config.location", System.getProperty("user.dir") + "/rpm/resources/application.properties")); + .get(System.getProperty("cfe_39.config.location", "/opt/teragrep/cfe_39/etc/application.properties")); LOGGER.info("Loading application config <[{}]>", configPath.toAbsolutePath()); try (InputStream inputStream = Files.newInputStream(configPath)) { @@ -98,10 +98,16 @@ public Config() throws IOException { // HDFS pruning this.pruneOffset = Long.parseLong(properties.getProperty("pruneOffset", "172800000")); + if (this.pruneOffset <= 0) { + throw new IllegalArgumentException("pruneOffset must be set to >0, got " + pruneOffset); + } // AVRO this.queueDirectory = properties.getProperty("queueDirectory", System.getProperty("user.dir") + "/etc/AVRO/"); this.maximumFileSize = Long.parseLong(properties.getProperty("maximumFileSize", "60800000")); + if (this.maximumFileSize <= 0) { + throw new IllegalArgumentException("maximumFileSize must be set to >0, got " + maximumFileSize); + } // kerberos this.kerberosHost = properties.getProperty("java.security.krb5.kdc"); From 5a989b0ad31d4e8324d4e614bb85b148d82c72eb Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Thu, 4 Jul 2024 10:06:06 +0300 Subject: [PATCH 131/146] Updated README.adoc and example config. --- README.adoc | 8 +++++--- rpm/resources/application.properties | 8 ++++---- 2 files changed, 9 insertions(+), 7 deletions(-) diff --git a/README.adoc b/README.adoc index 53bbf85b..4bd6c77a 100644 --- a/README.adoc +++ b/README.adoc @@ -13,11 +13,13 @@ See the official documentation on https://docs.teragrep.com[docs.teragrep.com]. ## How to [compile/use/implement] -// TODO: add instructions how people can start to use your project, also add more information on the required configuration files. -`mvn clean install` +`mvn clean package` -config.jaas, application.properties and log4j2.properties files have to created to use this module. The files must be placed in the working directory. +application.properties, config.jaas and log4j2.properties files have to be created to use this module. +By default, application.properties file must be placed in /opt/teragrep/cfe_39/etc/ directory. +The application.properties is used to define the directory where the other files must be placed. +Example configuration files available in cfe_39/rpm/resources/ directory. ## Contributing diff --git a/rpm/resources/application.properties b/rpm/resources/application.properties index cdeff44e..32e2b8f4 100644 --- a/rpm/resources/application.properties +++ b/rpm/resources/application.properties @@ -26,12 +26,12 @@ consumer.request.timeout.ms=300000 consumer.max.poll.interval.ms=300000 # For testing only, remove for prod. consumer.useMockKafkaConsumer=true -# AVRO +# Directory where AVRO files are constructed for HDFS queueDirectory=/opt/teragrep/cfe_39/etc/AVRO/ # The maximum file size for AVRO-files that are to be stored in HDFS database. -maximumFileSize=3000 -# HDFS pruning, use 157784760000 value while testing HDFS writes to ensure the test records are not pruned. 157784760000L -pruneOffset=157784760000 +maximumFileSize=60800000 +# HDFS pruning offset, prunes files older than the given milliseconds. +pruneOffset=172800000 # HDFS uri hdfsuri=hdfs://localhost:45937/ # Kerberos From 598cafa5049b7234887d8ba33834ba9798a1bbbc Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Thu, 4 Jul 2024 11:43:31 +0300 Subject: [PATCH 132/146] Added rat plugin. Fixed workflow runs-on parameter. Added disable condition to ingestion tests. --- .github/workflows/upload_release.yaml | 2 +- pom.xml | 65 +++++++++++++++++++ .../teragrep/cfe_39/Ingestion0FilesTest.java | 9 +++ .../cfe_39/Ingestion1Old1NewFileTest.java | 5 ++ .../cfe_39/Ingestion2NewFilesTest.java | 5 ++ .../cfe_39/Ingestion2OldFilesTest.java | 5 ++ 6 files changed, 90 insertions(+), 1 deletion(-) diff --git a/.github/workflows/upload_release.yaml b/.github/workflows/upload_release.yaml index 6c14eeca..4138e06f 100644 --- a/.github/workflows/upload_release.yaml +++ b/.github/workflows/upload_release.yaml @@ -7,7 +7,7 @@ on: jobs: upload: name: Upload - runs-on: ubuntu- + runs-on: ubuntu-latest permissions: contents: write diff --git a/pom.xml b/pom.xml index b2220cb8..ade85dca 100644 --- a/pom.xml +++ b/pom.xml @@ -148,6 +148,71 @@ ${project.artifactId}-${revision}${changelist}${sha1} + + org.apache.rat + apache-rat-plugin + 0.16.1 + false + + false + false + + + Teragrep + Affero General Public License v3 + + + + + + + Suomen Kanuuna Oy + 2024 + + HDFS Data Ingestion for PTH_06 use CFE-39 + + Teragrep + + + true + false + + + .git/** + .gitattributes + .gitignore + .gitmodules + + .github/workflows/* + .github/ISSUE_TEMPLATE/* + toolchains.xml + settings.xml + + pom.xml + eclipse-java-formatter.xml + + README.adoc + + license-header + src/main/avro/KafkaRecord.avsc + src/main/assembly/jar-with-dependencies.xml + src/test/resources/broken.application.properties + src/test/resources/valid.application.properties + rpm/resources/config.jaas + rpm/resources/log4j2.properties + rpm/resources/application.properties + rpm/rpm.pom.xml + + + + + + check + + test + + + org.apache.avro avro-maven-plugin diff --git a/src/test/java/com/teragrep/cfe_39/Ingestion0FilesTest.java b/src/test/java/com/teragrep/cfe_39/Ingestion0FilesTest.java index 23c52a85..cf2fe882 100644 --- a/src/test/java/com/teragrep/cfe_39/Ingestion0FilesTest.java +++ b/src/test/java/com/teragrep/cfe_39/Ingestion0FilesTest.java @@ -53,6 +53,7 @@ import org.apache.hadoop.fs.*; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.junit.jupiter.api.*; +import org.junit.jupiter.api.condition.DisabledIfSystemProperty; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.fs.Path; @@ -97,6 +98,10 @@ public void teardownMiniCluster() { FileUtil.fullyDelete(baseDir); } + @DisabledIfSystemProperty( + named = "skipIngestionTest", + matches = "true" + ) @Test public void ingestion0FilesTest() { /*This test case is for testing the functionality of the ingestion when there are no files already present in the database before starting ingestion. @@ -296,6 +301,10 @@ record = reader.next(record); }); } + @DisabledIfSystemProperty( + named = "skipIngestionTest", + matches = "true" + ) @Test public void ingestion0FilesLowSizeTest() { /*This test case is for testing the functionality of the ingestion when there are files already present in the database before starting ingestion. diff --git a/src/test/java/com/teragrep/cfe_39/Ingestion1Old1NewFileTest.java b/src/test/java/com/teragrep/cfe_39/Ingestion1Old1NewFileTest.java index 1ae2b380..79174e47 100644 --- a/src/test/java/com/teragrep/cfe_39/Ingestion1Old1NewFileTest.java +++ b/src/test/java/com/teragrep/cfe_39/Ingestion1Old1NewFileTest.java @@ -54,6 +54,7 @@ import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.condition.DisabledIfSystemProperty; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -129,6 +130,10 @@ public void teardownMiniCluster() { FileUtil.fullyDelete(baseDir); } + @DisabledIfSystemProperty( + named = "skipIngestionTest", + matches = "true" + ) @Test public void ingestion1Old1NewFileTest() { /* This test case is for testing the functionality of the ingestion when there are files already present in the database before starting ingestion. diff --git a/src/test/java/com/teragrep/cfe_39/Ingestion2NewFilesTest.java b/src/test/java/com/teragrep/cfe_39/Ingestion2NewFilesTest.java index 7eaca4f9..7c8e7db8 100644 --- a/src/test/java/com/teragrep/cfe_39/Ingestion2NewFilesTest.java +++ b/src/test/java/com/teragrep/cfe_39/Ingestion2NewFilesTest.java @@ -56,6 +56,7 @@ import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.condition.DisabledIfSystemProperty; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -132,6 +133,10 @@ public void teardownMiniCluster() { FileUtil.fullyDelete(baseDir); } + @DisabledIfSystemProperty( + named = "skipIngestionTest", + matches = "true" + ) @Test public void ingestion2NewFilesTest() { /* This test case is for testing the functionality of the ingestion when there are files already present in the database before starting ingestion. diff --git a/src/test/java/com/teragrep/cfe_39/Ingestion2OldFilesTest.java b/src/test/java/com/teragrep/cfe_39/Ingestion2OldFilesTest.java index b9e110b4..4424918b 100644 --- a/src/test/java/com/teragrep/cfe_39/Ingestion2OldFilesTest.java +++ b/src/test/java/com/teragrep/cfe_39/Ingestion2OldFilesTest.java @@ -54,6 +54,7 @@ import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.condition.DisabledIfSystemProperty; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -130,6 +131,10 @@ public void teardownMiniCluster() { FileUtil.fullyDelete(baseDir); } + @DisabledIfSystemProperty( + named = "skipIngestionTest", + matches = "true" + ) @Test public void ingestion2OldFilesTest() { /* This test case is for testing the functionality of the ingestion when there are files already present in the database before starting ingestion. From 882f233fe54ce9adc81cc65be1faefcd85ff8906 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Thu, 4 Jul 2024 12:12:50 +0300 Subject: [PATCH 133/146] Downgraded spotless to 2.30.0 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index ade85dca..2f0ec2db 100644 --- a/pom.xml +++ b/pom.xml @@ -305,7 +305,7 @@ com.diffplug.spotless spotless-maven-plugin - 2.43.0 + 2.30.0 From 38a0e383550bffffe2eb45a2d99270f4cd6394ef Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Thu, 4 Jul 2024 12:12:50 +0300 Subject: [PATCH 134/146] Downgraded spotless to 2.30.0 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index ade85dca..2f0ec2db 100644 --- a/pom.xml +++ b/pom.xml @@ -305,7 +305,7 @@ com.diffplug.spotless spotless-maven-plugin - 2.43.0 + 2.30.0 From dfdd03453ad230615cbc1dfcdc48b45e39fdd421 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Thu, 4 Jul 2024 12:53:16 +0300 Subject: [PATCH 135/146] Added exclusion for AVRO-generated file. --- pom.xml | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/pom.xml b/pom.xml index 2f0ec2db..ecd7409d 100644 --- a/pom.xml +++ b/pom.xml @@ -202,6 +202,7 @@ rpm/resources/log4j2.properties rpm/resources/application.properties rpm/rpm.pom.xml + src/main/java/com/teragrep/cfe_39/avro/SyslogRecord.java @@ -316,6 +317,9 @@ ${project.basedir}/license-header + + src/main/java/com/teragrep/cfe_39/avro/SyslogRecord.java + From 7411708f5ef8f3b8619cbdeb41e50fe63add2368 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Thu, 4 Jul 2024 13:00:28 +0300 Subject: [PATCH 136/146] Added exclusion to .gitignore --- .gitignore | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.gitignore b/.gitignore index 2f435308..cc5ebc0c 100644 --- a/.gitignore +++ b/.gitignore @@ -15,3 +15,5 @@ buildNumber.properties .project # JDT-specific (Eclipse Java Development Tools) .classpath + +src/main/java/com/teragrep/cfe_39/avro/SyslogRecord.java \ No newline at end of file From 1307439d75c512b4a85b7b168c31832f040224a0 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Thu, 4 Jul 2024 12:53:16 +0300 Subject: [PATCH 137/146] Added exclusion for AVRO-generated file. --- pom.xml | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/pom.xml b/pom.xml index 2f0ec2db..ecd7409d 100644 --- a/pom.xml +++ b/pom.xml @@ -202,6 +202,7 @@ rpm/resources/log4j2.properties rpm/resources/application.properties rpm/rpm.pom.xml + src/main/java/com/teragrep/cfe_39/avro/SyslogRecord.java @@ -316,6 +317,9 @@ ${project.basedir}/license-header + + src/main/java/com/teragrep/cfe_39/avro/SyslogRecord.java + From 578ddb0b772710af3cfdfdf715cf2808e94f4bd0 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Thu, 4 Jul 2024 13:00:28 +0300 Subject: [PATCH 138/146] Added exclusion to .gitignore --- .gitignore | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.gitignore b/.gitignore index 2f435308..cc5ebc0c 100644 --- a/.gitignore +++ b/.gitignore @@ -15,3 +15,5 @@ buildNumber.properties .project # JDT-specific (Eclipse Java Development Tools) .classpath + +src/main/java/com/teragrep/cfe_39/avro/SyslogRecord.java \ No newline at end of file From c4e122f3f1ba73f0708897dd6b5220621a4e93a4 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Thu, 4 Jul 2024 13:07:13 +0300 Subject: [PATCH 139/146] Fix to rpm packaging configuration --- rpm/rpm.pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/rpm/rpm.pom.xml b/rpm/rpm.pom.xml index 2eefdddc..eaf5c2b7 100644 --- a/rpm/rpm.pom.xml +++ b/rpm/rpm.pom.xml @@ -5,7 +5,7 @@ rpm 4.0.0 cfe_39 - ${revision}${sha1}${changelist}.${env.TARGET_OS}.x86_64 + ${revision}${sha1}${changelist} cfe_39 cfe_39 com.teragrep From 221bfdecf03d0dd48ba024a7308a28cf0ffdb537 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Thu, 4 Jul 2024 13:40:15 +0300 Subject: [PATCH 140/146] Added configuration files to rpm packaging. --- rpm/rpm.pom.xml | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/rpm/rpm.pom.xml b/rpm/rpm.pom.xml index eaf5c2b7..fe7b95c6 100644 --- a/rpm/rpm.pom.xml +++ b/rpm/rpm.pom.xml @@ -94,6 +94,22 @@ + + /opt/teragrep/${project.artifactId}/etc + true + noreplace + + + ${project.basedir}/../rpm/resources/application.properties + + + ${project.basedir}/../rpm/resources/config.jaas + + + ${project.basedir}/../rpm/resources/log4j2.properties + + + From 0f50aa7325878b41f4b885231bddafe39304183a Mon Sep 17 00:00:00 2001 From: StrongestNumber9 <16169054+StrongestNumber9@users.noreply.github.com> Date: Thu, 4 Jul 2024 15:11:05 +0300 Subject: [PATCH 141/146] Adds systemd service file, new service user to rpm --- pom.xml | 1 + rpm/resources/cfe_39.service | 12 ++++++++++++ rpm/rpm.pom.xml | 29 ++++++++++++++++++++++++----- 3 files changed, 37 insertions(+), 5 deletions(-) create mode 100644 rpm/resources/cfe_39.service diff --git a/pom.xml b/pom.xml index ecd7409d..535a8be9 100644 --- a/pom.xml +++ b/pom.xml @@ -201,6 +201,7 @@ rpm/resources/config.jaas rpm/resources/log4j2.properties rpm/resources/application.properties + rpm/resources/cfe_39.service rpm/rpm.pom.xml src/main/java/com/teragrep/cfe_39/avro/SyslogRecord.java diff --git a/rpm/resources/cfe_39.service b/rpm/resources/cfe_39.service new file mode 100644 index 00000000..6bb58d56 --- /dev/null +++ b/rpm/resources/cfe_39.service @@ -0,0 +1,12 @@ +[Unit] +Description=com.teragrep.cfe_39 +ConditionPathExists=/opt/teragrep/cfe_39/lib/cfe_39.jar + +[Service] +ExecStart=/usr/lib/jvm/jre-1.8.0-openjdk/bin/java -jar /opt/teragrep/cfe_39/lib/cfe_39.jar +User=srv-cfe_39 +WorkingDirectory=/opt/teragrep/cfe_39 + +[Install] +WantedBy=multi-user.target + diff --git a/rpm/rpm.pom.xml b/rpm/rpm.pom.xml index fe7b95c6..fa4fb42a 100644 --- a/rpm/rpm.pom.xml +++ b/rpm/rpm.pom.xml @@ -69,8 +69,8 @@ teragrep <servicedesk@teragrep.com> teragrep/LogManagementSuite false - root - root + srv-cfe_39 + srv-cfe_39 0644 0755 @@ -84,8 +84,8 @@ true 755 755 - root - root + srv-cfe_39 + srv-cfe_39 true @@ -110,7 +110,26 @@ + + /usr/lib/systemd/system + false + + + ${project.basedir}/resources/cfe_39.service + + + + + java-1.8.0-openjdk + + + + @@ -140,4 +159,4 @@ - \ No newline at end of file + From a44b70ed6f71f4148d6a2d0cecdb4589cef3cc8b Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Tue, 6 Aug 2024 15:55:29 +0300 Subject: [PATCH 142/146] Fix for issue #32 by enabling automatic TGT renewal. --- src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java | 1 + .../com/teragrep/cfe_39/consumers/kafka/HdfsDataIngestion.java | 1 + 2 files changed, 2 insertions(+) diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java index d5ad57f8..368c84f2 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java @@ -120,6 +120,7 @@ These values should be fetched from config and other input parameters (topic+par // enable kerberus conf.set("hadoop.security.authentication", config.getHadoopAuthentication()); conf.set("hadoop.security.authorization", config.getHadoopAuthorization()); + conf.set("hadoop.kerberos.keytab.login.autorenewal.enabled", "true"); conf.set("fs.defaultFS", hdfsuri); // Set FileSystem URI conf.set("fs.hdfs.impl", DistributedFileSystem.class.getName()); // Maven stuff? diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HdfsDataIngestion.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HdfsDataIngestion.java index 460f9448..233ccca0 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HdfsDataIngestion.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HdfsDataIngestion.java @@ -123,6 +123,7 @@ public HdfsDataIngestion(Config config) throws IOException { // enable kerberus conf.set("hadoop.security.authentication", config.getHadoopAuthentication()); conf.set("hadoop.security.authorization", config.getHadoopAuthorization()); + conf.set("hadoop.kerberos.keytab.login.autorenewal.enabled", "true"); conf.set("fs.defaultFS", hdfsuri); // Set FileSystem URI conf.set("fs.hdfs.impl", DistributedFileSystem.class.getName()); // Maven stuff? conf.set("fs.file.impl", LocalFileSystem.class.getName()); // Maven stuff? From 2742be9cbb855abe778306b8a1a2da9852c34c0b Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Wed, 7 Aug 2024 13:09:21 +0300 Subject: [PATCH 143/146] Replaced hadoop Configuration class usage with the child class HdfsConfiguration. --- .../com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java | 8 ++++---- .../cfe_39/consumers/kafka/HdfsDataIngestion.java | 6 +++--- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java index 368c84f2..9c7a5452 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java @@ -47,9 +47,9 @@ import com.google.gson.JsonObject; import com.teragrep.cfe_39.Config; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.*; import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.security.UserGroupInformation; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -66,7 +66,7 @@ public class HDFSWrite implements AutoCloseable { private final String path; private final FileSystem fs; private final boolean useMockKafkaConsumer; // Defines if mock HDFS database is used for testing - private final Configuration conf; + private final HdfsConfiguration conf; private final String hdfsuri; public HDFSWrite(Config config, JsonObject lastObjectJo) throws IOException { @@ -86,7 +86,7 @@ These values should be fetched from config and other input parameters (topic+par fileName = lastObjectJo.get("partition").getAsString() + "." + lastObjectJo.get("offset").getAsString(); // filename should be constructed from partition and offset. // ====== Init HDFS File System Object - conf = new Configuration(); + conf = new HdfsConfiguration(); // Set FileSystem URI conf.set("fs.defaultFS", hdfsuri); // Because of Maven @@ -115,7 +115,7 @@ These values should be fetched from config and other input parameters (topic+par System.setProperty("java.security.krb5.realm", config.getKerberosRealm()); System.setProperty("java.security.krb5.kdc", config.getKerberosHost()); - conf = new Configuration(); + conf = new HdfsConfiguration(); // enable kerberus conf.set("hadoop.security.authentication", config.getHadoopAuthentication()); diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HdfsDataIngestion.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HdfsDataIngestion.java index 233ccca0..6c7429c9 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HdfsDataIngestion.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HdfsDataIngestion.java @@ -48,10 +48,10 @@ import com.teragrep.cfe_39.Config; import com.teragrep.cfe_39.metrics.*; import com.teragrep.cfe_39.metrics.topic.TopicCounter; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.LocalFileSystem; import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.security.UserGroupInformation; import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.common.PartitionInfo; @@ -96,7 +96,7 @@ public HdfsDataIngestion(Config config) throws IOException { // Initializing the FileSystem with minicluster. String hdfsuri = config.getHdfsuri(); // ====== Init HDFS File System Object - Configuration conf = new Configuration(); + HdfsConfiguration conf = new HdfsConfiguration(); // Set FileSystem URI conf.set("fs.defaultFS", hdfsuri); // Because of Maven @@ -119,7 +119,7 @@ public HdfsDataIngestion(Config config) throws IOException { // set kerberos host and realm System.setProperty("java.security.krb5.realm", config.getKerberosRealm()); System.setProperty("java.security.krb5.kdc", config.getKerberosHost()); - Configuration conf = new Configuration(); + HdfsConfiguration conf = new HdfsConfiguration(); // enable kerberus conf.set("hadoop.security.authentication", config.getHadoopAuthentication()); conf.set("hadoop.security.authorization", config.getHadoopAuthorization()); From 60cc06ca5fbe05aaa5a79272e3a71bbe76626359 Mon Sep 17 00:00:00 2001 From: Tiihott <119838215+Tiihott@users.noreply.github.com> Date: Tue, 20 Aug 2024 15:12:05 +0300 Subject: [PATCH 144/146] Added handling of NULL records and un-parseable records. (#35) * Fix for issue #32 by enabling automatic TGT renewal. * Replaced hadoop Configuration class usage with the child class HdfsConfiguration. * Added simplified handling for consumed kafka records with null content. * Added additional null content records to MockKafkaConsumerFactory for tests. * Added exception handling for non-parseable records. Modified test data and tests accordingly. * Moved NULL record handling from KafkaReader to DatabaseOutput. Changed KafkaConsumerTest accordingly. * Added configuration flags for skipping empty and non RFC5424 records. Improved logging to give metadata for records causing the exceptions. * Added configuration flags for automatic kerberos TGT renewal. * Added separate record processing tests. Added cleanup of temporary files that weren't written to HDFS because of exception during processing. * Added checkTGTAndReloginFromKeytab() functionality. --------- Co-authored-by: Mikko Kortelainen --- pom.xml | 1 + rpm/resources/application.properties | 7 +- src/main/java/com/teragrep/cfe_39/Config.java | 27 ++ .../consumers/kafka/DatabaseOutput.java | 52 ++- .../cfe_39/consumers/kafka/HDFSWrite.java | 6 +- .../consumers/kafka/HdfsDataIngestion.java | 7 +- .../kafka/MockKafkaConsumerFactory.java | 21 ++ .../teragrep/cfe_39/KafkaConsumerTest.java | 241 +++++++++++++- .../cfe_39/ProcessingFailureTest.java | 194 +++++++++++ .../com/teragrep/cfe_39/ProcessingTest.java | 311 ++++++++++++++++++ .../resources/broken.application.properties | 7 +- .../failProcessing.application.properties | 44 +++ .../resources/valid.application.properties | 7 +- 13 files changed, 906 insertions(+), 19 deletions(-) create mode 100644 src/test/java/com/teragrep/cfe_39/ProcessingFailureTest.java create mode 100644 src/test/java/com/teragrep/cfe_39/ProcessingTest.java create mode 100644 src/test/resources/failProcessing.application.properties diff --git a/pom.xml b/pom.xml index 535a8be9..bda84206 100644 --- a/pom.xml +++ b/pom.xml @@ -198,6 +198,7 @@ src/main/assembly/jar-with-dependencies.xml src/test/resources/broken.application.properties src/test/resources/valid.application.properties + src/test/resources/failProcessing.application.properties rpm/resources/config.jaas rpm/resources/log4j2.properties rpm/resources/application.properties diff --git a/rpm/resources/application.properties b/rpm/resources/application.properties index 32e2b8f4..06ab6c7c 100644 --- a/rpm/resources/application.properties +++ b/rpm/resources/application.properties @@ -30,6 +30,10 @@ consumer.useMockKafkaConsumer=true queueDirectory=/opt/teragrep/cfe_39/etc/AVRO/ # The maximum file size for AVRO-files that are to be stored in HDFS database. maximumFileSize=60800000 +# Boolean for deciding if records not in RFC5424 should be skipped or not. +skipNonRFC5424Records=true +# Boolean for deciding if empty RFC5424 records should be skipped or not. +skipEmptyRFC5424Records=true # HDFS pruning offset, prunes files older than the given milliseconds. pruneOffset=172800000 # HDFS uri @@ -42,4 +46,5 @@ hadoop.security.authorization=test dfs.namenode.kerberos.principal.pattern=test KerberosKeytabUser=test KerberosKeytabPath=test -dfs.client.use.datanode.hostname=false \ No newline at end of file +dfs.client.use.datanode.hostname=false +kerberosLoginAutorenewal=true \ No newline at end of file diff --git a/src/main/java/com/teragrep/cfe_39/Config.java b/src/main/java/com/teragrep/cfe_39/Config.java index 65dad468..24d43b3d 100644 --- a/src/main/java/com/teragrep/cfe_39/Config.java +++ b/src/main/java/com/teragrep/cfe_39/Config.java @@ -73,10 +73,13 @@ public class Config { private final String hadoopAuthorization; private final String kerberosKeytabUser; private final String kerberosKeytabPath; + private final String kerberosLoginAutorenewal; private final String kerberosTestMode; private long maximumFileSize; private final int numOfConsumers; private final long pruneOffset; + private final boolean skipNonRFC5424Records; + private final boolean skipEmptyRFC5424Records; public Config() throws IOException { Properties properties = new Properties(); @@ -138,12 +141,24 @@ public Config() throws IOException { if (this.kerberosKeytabPath == null) { throw new IllegalArgumentException("kerberosKeytabPath not set"); } + this.kerberosLoginAutorenewal = properties.getProperty("kerberosLoginAutorenewal"); + if (this.kerberosLoginAutorenewal == null) { + throw new IllegalArgumentException("kerberosLoginAutorenewal not set"); + } this.kerberosTestMode = properties.getProperty("dfs.client.use.datanode.hostname", "false"); // kafka this.queueTopicPattern = properties.getProperty("queueTopicPattern", "^.*$"); this.numOfConsumers = Integer.parseInt(properties.getProperty("numOfConsumers", "1")); + // skip non RFC5424 records + this.skipNonRFC5424Records = properties.getProperty("skipNonRFC5424Records", "false").equalsIgnoreCase("true"); + + // skip empty RFC5424 records + this.skipEmptyRFC5424Records = properties + .getProperty("skipEmptyRFC5424Records", "false") + .equalsIgnoreCase("true"); + this.kafkaConsumerProperties = loadSubProperties(properties, "consumer."); String loginConfig = properties .getProperty("java.security.auth.login.config", System.getProperty("user.dir") + "/rpm/resources/config.jaas"); @@ -247,4 +262,16 @@ public int getNumOfConsumers() { public long getPruneOffset() { return pruneOffset; } + + public boolean getSkipNonRFC5424Records() { + return skipNonRFC5424Records; + } + + public boolean getSkipEmptyRFC5424Records() { + return skipEmptyRFC5424Records; + } + + public String getKerberosLoginAutorenewal() { + return kerberosLoginAutorenewal; + } } diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java index b82f5111..ae519335 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/DatabaseOutput.java @@ -95,6 +95,8 @@ public class DatabaseOutput implements Consumer> { private final SDVector originHostname; private File syslogFile; private final Config config; + private final boolean skipNonRFC5424Records; + private final boolean skipEmptyRFC5424Records; public DatabaseOutput( Config config, @@ -121,6 +123,8 @@ public DatabaseOutput( this.eventNodeSourceHostname = new SDVector("event_node_source@48577", "hostname"); this.eventNodeRelayHostname = new SDVector("event_node_relay@48577", "hostname"); this.originHostname = new SDVector("origin@48577", "hostname"); + this.skipNonRFC5424Records = config.getSkipNonRFC5424Records(); + this.skipEmptyRFC5424Records = config.getSkipEmptyRFC5424Records(); } // Checks that the filesize stays under the defined maximum file size. If the file is about to go over target limit commits the file to HDFS and returns true, otherwise does nothing and returns false. @@ -254,16 +258,37 @@ public void accept(List recordOffsetObjectList) { } byte[] byteArray = recordOffsetObject.getRecord(); // loads the byte[] contained in recordOffsetObject.getRecord() to byteArray. - batchBytes = batchBytes + byteArray.length; + if (byteArray == null) { + if (skipEmptyRFC5424Records) { + if (LOGGER.isDebugEnabled()) { + LOGGER + .debug( + "Skipping processing an empty non RFC5424 record. Record metadata: {}", + recordOffsetObject.offsetToJSON() + ); + } + continue; + } + else { + if (LOGGER.isDebugEnabled()) { + LOGGER.debug("Null record metadata: {}", recordOffsetObject.offsetToJSON()); + } + syslogFile.delete(); // Clean up + throw new NullPointerException("Record with null content detected during processing."); + } + + } InputStream inputStream = new ByteArrayInputStream(byteArray); rfc5424Frame.load(inputStream); try { if (rfc5424Frame.next()) { - /* rfc5424Frame has loaded the record data, it's ready for deserialization. + /*rfc5424Frame has loaded the record data, it's ready for deserialization. Implement AVRO serialization for the Kafka records here, preparing the data for writing to HDFS. Write all the data into a file using AVRO. The size of each AVRO-serialized file should be as close to 64M as possible.*/ + batchBytes = batchBytes + byteArray.length; + // input final byte[] source = eventToSource(); @@ -323,6 +348,29 @@ public void accept(List recordOffsetObjectList) { catch (IOException e) { throw new UncheckedIOException(e); } + catch (ParseException e) { + if (skipNonRFC5424Records) { + if (LOGGER.isDebugEnabled()) { + LOGGER + .debug( + "Skipping processing a non RFC5424 record, record metadata: {}. Exception information: ", + recordOffsetObject.offsetToJSON(), e + ); + } + continue; + } + else { + if (LOGGER.isDebugEnabled()) { + LOGGER + .debug( + "Record metadata that is causing ParseException: {}.", + recordOffsetObject.offsetToJSON() + ); + } + syslogFile.delete(); // Clean up + throw new RuntimeException(e); + } + } } // Handle the "leftover" syslogRecords from the loop. diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java index 9c7a5452..ab5c9632 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java @@ -120,7 +120,7 @@ These values should be fetched from config and other input parameters (topic+par // enable kerberus conf.set("hadoop.security.authentication", config.getHadoopAuthentication()); conf.set("hadoop.security.authorization", config.getHadoopAuthorization()); - conf.set("hadoop.kerberos.keytab.login.autorenewal.enabled", "true"); + conf.set("hadoop.kerberos.keytab.login.autorenewal.enabled", config.getKerberosLoginAutorenewal()); conf.set("fs.defaultFS", hdfsuri); // Set FileSystem URI conf.set("fs.hdfs.impl", DistributedFileSystem.class.getName()); // Maven stuff? @@ -132,10 +132,6 @@ These values should be fetched from config and other input parameters (topic+par // server principal, the kerberos principle that the namenode is using conf.set("dfs.namenode.kerberos.principal.pattern", config.getKerberosPrincipal()); - // set usergroup stuff - UserGroupInformation.setConfiguration(conf); - UserGroupInformation.loginUserFromKeytab(config.getKerberosKeytabUser(), config.getKerberosKeytabPath()); - // filesystem for HDFS access is set here fs = FileSystem.get(conf); } diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HdfsDataIngestion.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HdfsDataIngestion.java index 6c7429c9..d710a8dd 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HdfsDataIngestion.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HdfsDataIngestion.java @@ -123,7 +123,7 @@ public HdfsDataIngestion(Config config) throws IOException { // enable kerberus conf.set("hadoop.security.authentication", config.getHadoopAuthentication()); conf.set("hadoop.security.authorization", config.getHadoopAuthorization()); - conf.set("hadoop.kerberos.keytab.login.autorenewal.enabled", "true"); + conf.set("hadoop.kerberos.keytab.login.autorenewal.enabled", config.getKerberosLoginAutorenewal()); conf.set("fs.defaultFS", hdfsuri); // Set FileSystem URI conf.set("fs.hdfs.impl", DistributedFileSystem.class.getName()); // Maven stuff? conf.set("fs.file.impl", LocalFileSystem.class.getName()); // Maven stuff? @@ -142,7 +142,7 @@ public HdfsDataIngestion(Config config) throws IOException { hdfsStartOffsets = new HashMap<>(); } - public void run() throws InterruptedException { + public void run() throws InterruptedException, IOException { // Initialize and register duration statistics DurationStatistics durationStatistics = new DurationStatistics(); @@ -161,6 +161,9 @@ public void run() throws InterruptedException { } while (keepRunning) { + if ("kerberos".equals(config.getHadoopAuthentication())) { + UserGroupInformation.getLoginUser().checkTGTAndReloginFromKeytab(); + } LOGGER.debug("Scanning for threads"); topicScan(durationStatistics, topicCounters); diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactory.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactory.java index a5223b17..e5da3a81 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactory.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/MockKafkaConsumerFactory.java @@ -228,6 +228,27 @@ private static void generateEvents(MockConsumer consumer, String .getBytes(StandardCharsets.UTF_8) ) ); + consumer + .addRecord( + new ConsumerRecord<>( + topicName, + partition, + 14L, + "2022-04-25T07:34:52.244Z".getBytes(StandardCharsets.UTF_8), + null + ) + ); + consumer + .addRecord( + new ConsumerRecord<>( + topicName, + partition, + 15L, + "2022-04-25T07:34:52.245Z".getBytes(StandardCharsets.UTF_8), + "12>1 2022-04-25T07:34:52.245Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"3bb55ce4-0ea7-413a-b403-28b174d7ac99\" source=\"source\" unixtime=\"1650872092243\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872092\"] 25.04.2022 07:34:52.243 [ERROR] com.teragrep.jla_02.Log4j2 [instanceId=01, thread=Thread-0, userId=, sessionId=, requestId=, SUBJECT=, VERB=, OBJECT=, OUTCOME=, message=Log4j2 error metric says hi!]" + .getBytes(StandardCharsets.UTF_8) + ) + ); } // Can initialize topic scan with all partitions available when the input parameter is 0. Consumer is manually assigned to specific partitions depending on the threadnum parameter. For example on threadnum 1 consumer has odd numbered partitions assigned to it and threadnum 2 has the even numbered partitions. diff --git a/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java b/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java index 593794be..0087786a 100644 --- a/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java +++ b/src/test/java/com/teragrep/cfe_39/KafkaConsumerTest.java @@ -47,6 +47,7 @@ import com.teragrep.cfe_39.consumers.kafka.ReadCoordinator; import com.teragrep.cfe_39.consumers.kafka.RecordOffset; +import com.teragrep.rlo_06.ParseException; import com.teragrep.rlo_06.RFC5424Frame; import org.apache.kafka.common.TopicPartition; import org.junit.jupiter.api.Assertions; @@ -97,9 +98,9 @@ public void readCoordinatorTest2Threads() { Thread.sleep(10000); Assertions.assertEquals(2, messages.size()); - Assertions.assertEquals(140, messages.get(0).size() + messages.get(1).size()); // Assert that expected amount of records has been consumed by the consumer group. - Assertions.assertEquals(70, messages.get(0).size()); - Assertions.assertEquals(70, messages.get(1).size()); + Assertions.assertEquals(160, messages.get(0).size() + messages.get(1).size()); // Assert that expected amount of records has been consumed by the consumer group. + Assertions.assertEquals(80, messages.get(0).size()); + Assertions.assertEquals(80, messages.get(1).size()); // Assert that all the record contents are correct, every topic partition has identical set of offset-message pairings. List messageList = new ArrayList(); @@ -166,6 +167,26 @@ public void readCoordinatorTest2Threads() { counter++; } + recordOffset = messages.get(0).get(counter); + Assertions + .assertEquals( + "{\"topic\":\"testConsumerTopic\", \"partition\":7, \"offset\":" + 14 + "}", + recordOffset.offsetToJSON() + ); + Assertions.assertNull(recordOffset.getRecord()); + counter++; + + recordOffset = messages.get(0).get(counter); + Assertions + .assertEquals( + "{\"topic\":\"testConsumerTopic\", \"partition\":7, \"offset\":" + 15 + "}", + recordOffset.offsetToJSON() + ); + rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); + ParseException e = Assertions.assertThrows(ParseException.class, rfc5424Frame::next); + Assertions.assertEquals("PRIORITY < missing", e.getMessage()); + counter++; + iterator = messageList.iterator(); for (int i = 0; i <= 13; i++) { recordOffset = messages.get(0).get(counter); @@ -182,6 +203,26 @@ public void readCoordinatorTest2Threads() { counter++; } + recordOffset = messages.get(0).get(counter); + Assertions + .assertEquals( + "{\"topic\":\"testConsumerTopic\", \"partition\":5, \"offset\":" + 14 + "}", + recordOffset.offsetToJSON() + ); + Assertions.assertNull(recordOffset.getRecord()); + counter++; + + recordOffset = messages.get(0).get(counter); + Assertions + .assertEquals( + "{\"topic\":\"testConsumerTopic\", \"partition\":5, \"offset\":" + 15 + "}", + recordOffset.offsetToJSON() + ); + rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); + e = Assertions.assertThrows(ParseException.class, rfc5424Frame::next); + Assertions.assertEquals("PRIORITY < missing", e.getMessage()); + counter++; + iterator = messageList.iterator(); for (int i = 0; i <= 13; i++) { recordOffset = messages.get(0).get(counter); @@ -198,6 +239,26 @@ public void readCoordinatorTest2Threads() { counter++; } + recordOffset = messages.get(0).get(counter); + Assertions + .assertEquals( + "{\"topic\":\"testConsumerTopic\", \"partition\":3, \"offset\":" + 14 + "}", + recordOffset.offsetToJSON() + ); + Assertions.assertNull(recordOffset.getRecord()); + counter++; + + recordOffset = messages.get(0).get(counter); + Assertions + .assertEquals( + "{\"topic\":\"testConsumerTopic\", \"partition\":3, \"offset\":" + 15 + "}", + recordOffset.offsetToJSON() + ); + rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); + e = Assertions.assertThrows(ParseException.class, rfc5424Frame::next); + Assertions.assertEquals("PRIORITY < missing", e.getMessage()); + counter++; + iterator = messageList.iterator(); for (int i = 0; i <= 13; i++) { recordOffset = messages.get(0).get(counter); @@ -214,6 +275,26 @@ public void readCoordinatorTest2Threads() { counter++; } + recordOffset = messages.get(0).get(counter); + Assertions + .assertEquals( + "{\"topic\":\"testConsumerTopic\", \"partition\":1, \"offset\":" + 14 + "}", + recordOffset.offsetToJSON() + ); + Assertions.assertNull(recordOffset.getRecord()); + counter++; + + recordOffset = messages.get(0).get(counter); + Assertions + .assertEquals( + "{\"topic\":\"testConsumerTopic\", \"partition\":1, \"offset\":" + 15 + "}", + recordOffset.offsetToJSON() + ); + rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); + e = Assertions.assertThrows(ParseException.class, rfc5424Frame::next); + Assertions.assertEquals("PRIORITY < missing", e.getMessage()); + counter++; + iterator = messageList.iterator(); for (int i = 0; i <= 13; i++) { recordOffset = messages.get(0).get(counter); @@ -230,7 +311,27 @@ public void readCoordinatorTest2Threads() { counter++; } - Assertions.assertEquals(70, counter); + recordOffset = messages.get(0).get(counter); + Assertions + .assertEquals( + "{\"topic\":\"testConsumerTopic\", \"partition\":9, \"offset\":" + 14 + "}", + recordOffset.offsetToJSON() + ); + Assertions.assertNull(recordOffset.getRecord()); + counter++; + + recordOffset = messages.get(0).get(counter); + Assertions + .assertEquals( + "{\"topic\":\"testConsumerTopic\", \"partition\":9, \"offset\":" + 15 + "}", + recordOffset.offsetToJSON() + ); + rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); + e = Assertions.assertThrows(ParseException.class, rfc5424Frame::next); + Assertions.assertEquals("PRIORITY < missing", e.getMessage()); + counter++; + + Assertions.assertEquals(80, counter); counter = 0; iterator = messageList.iterator(); @@ -249,6 +350,26 @@ public void readCoordinatorTest2Threads() { counter++; } + recordOffset = messages.get(1).get(counter); + Assertions + .assertEquals( + "{\"topic\":\"testConsumerTopic\", \"partition\":8, \"offset\":" + 14 + "}", + recordOffset.offsetToJSON() + ); + Assertions.assertNull(recordOffset.getRecord()); + counter++; + + recordOffset = messages.get(1).get(counter); + Assertions + .assertEquals( + "{\"topic\":\"testConsumerTopic\", \"partition\":8, \"offset\":" + 15 + "}", + recordOffset.offsetToJSON() + ); + rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); + e = Assertions.assertThrows(ParseException.class, rfc5424Frame::next); + Assertions.assertEquals("PRIORITY < missing", e.getMessage()); + counter++; + iterator = messageList.iterator(); for (int i = 0; i <= 13; i++) { recordOffset = messages.get(1).get(counter); @@ -265,6 +386,26 @@ public void readCoordinatorTest2Threads() { counter++; } + recordOffset = messages.get(1).get(counter); + Assertions + .assertEquals( + "{\"topic\":\"testConsumerTopic\", \"partition\":6, \"offset\":" + 14 + "}", + recordOffset.offsetToJSON() + ); + Assertions.assertNull(recordOffset.getRecord()); + counter++; + + recordOffset = messages.get(1).get(counter); + Assertions + .assertEquals( + "{\"topic\":\"testConsumerTopic\", \"partition\":6, \"offset\":" + 15 + "}", + recordOffset.offsetToJSON() + ); + rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); + e = Assertions.assertThrows(ParseException.class, rfc5424Frame::next); + Assertions.assertEquals("PRIORITY < missing", e.getMessage()); + counter++; + iterator = messageList.iterator(); for (int i = 0; i <= 13; i++) { recordOffset = messages.get(1).get(counter); @@ -281,6 +422,26 @@ public void readCoordinatorTest2Threads() { counter++; } + recordOffset = messages.get(1).get(counter); + Assertions + .assertEquals( + "{\"topic\":\"testConsumerTopic\", \"partition\":4, \"offset\":" + 14 + "}", + recordOffset.offsetToJSON() + ); + Assertions.assertNull(recordOffset.getRecord()); + counter++; + + recordOffset = messages.get(1).get(counter); + Assertions + .assertEquals( + "{\"topic\":\"testConsumerTopic\", \"partition\":4, \"offset\":" + 15 + "}", + recordOffset.offsetToJSON() + ); + rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); + e = Assertions.assertThrows(ParseException.class, rfc5424Frame::next); + Assertions.assertEquals("PRIORITY < missing", e.getMessage()); + counter++; + iterator = messageList.iterator(); for (int i = 0; i <= 13; i++) { recordOffset = messages.get(1).get(counter); @@ -297,6 +458,26 @@ public void readCoordinatorTest2Threads() { counter++; } + recordOffset = messages.get(1).get(counter); + Assertions + .assertEquals( + "{\"topic\":\"testConsumerTopic\", \"partition\":2, \"offset\":" + 14 + "}", + recordOffset.offsetToJSON() + ); + Assertions.assertNull(recordOffset.getRecord()); + counter++; + + recordOffset = messages.get(1).get(counter); + Assertions + .assertEquals( + "{\"topic\":\"testConsumerTopic\", \"partition\":2, \"offset\":" + 15 + "}", + recordOffset.offsetToJSON() + ); + rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); + e = Assertions.assertThrows(ParseException.class, rfc5424Frame::next); + Assertions.assertEquals("PRIORITY < missing", e.getMessage()); + counter++; + iterator = messageList.iterator(); for (int i = 0; i <= 13; i++) { recordOffset = messages.get(1).get(counter); @@ -312,7 +493,28 @@ public void readCoordinatorTest2Threads() { Assertions.assertFalse(rfc5424Frame.next()); counter++; } - Assertions.assertEquals(70, counter); + + recordOffset = messages.get(1).get(counter); + Assertions + .assertEquals( + "{\"topic\":\"testConsumerTopic\", \"partition\":0, \"offset\":" + 14 + "}", + recordOffset.offsetToJSON() + ); + Assertions.assertNull(recordOffset.getRecord()); + counter++; + + recordOffset = messages.get(1).get(counter); + Assertions + .assertEquals( + "{\"topic\":\"testConsumerTopic\", \"partition\":0, \"offset\":" + 15 + "}", + recordOffset.offsetToJSON() + ); + rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); + e = Assertions.assertThrows(ParseException.class, rfc5424Frame::next); + Assertions.assertEquals("PRIORITY < missing", e.getMessage()); + counter++; + + Assertions.assertEquals(80, counter); }); } @@ -320,6 +522,9 @@ public void readCoordinatorTest2Threads() { @Test public void readCoordinatorTest1Thread() { assertDoesNotThrow(() -> { + // Set system properties to use the valid configuration. + System + .setProperty("cfe_39.config.location", System.getProperty("user.dir") + "/src/test/resources/valid.application.properties"); Config config = new Config(); Map hdfsStartOffsets = new HashMap<>(); ArrayList> messages = new ArrayList<>(); @@ -336,7 +541,7 @@ public void readCoordinatorTest1Thread() { Thread.sleep(10000); Assertions.assertEquals(1, messages.size()); - Assertions.assertEquals(140, messages.get(0).size()); // Assert that expected amount of records has been consumed by the consumer. + Assertions.assertEquals(160, messages.get(0).size()); // Assert that expected amount of records has been consumed by the consumer. // Assert that all the record contents are correct, every topic partition has identical set of offset-message pairings. List list = new ArrayList(); @@ -414,9 +619,31 @@ public void readCoordinatorTest1Thread() { Assertions.assertFalse(rfc5424Frame.next()); counter++; } + + recordOffset = messages.get(0).get(counter); + Assertions + .assertEquals( + "{\"topic\":\"testConsumerTopic\", \"partition\":" + partition + ", \"offset\":" + 14 + + "}", + recordOffset.offsetToJSON() + ); + Assertions.assertNull(recordOffset.getRecord()); + counter++; + + recordOffset = messages.get(0).get(counter); + Assertions + .assertEquals( + "{\"topic\":\"testConsumerTopic\", \"partition\":" + partition + ", \"offset\":" + 15 + + "}", + recordOffset.offsetToJSON() + ); + rfc5424Frame.load(new ByteArrayInputStream(recordOffset.getRecord())); + ParseException e = Assertions.assertThrows(ParseException.class, rfc5424Frame::next); + Assertions.assertEquals("PRIORITY < missing", e.getMessage()); + counter++; } - Assertions.assertEquals(140, counter); // All 140 records asserted. + Assertions.assertEquals(160, counter); // All 160 records were asserted. }); } diff --git a/src/test/java/com/teragrep/cfe_39/ProcessingFailureTest.java b/src/test/java/com/teragrep/cfe_39/ProcessingFailureTest.java new file mode 100644 index 00000000..63d83e8c --- /dev/null +++ b/src/test/java/com/teragrep/cfe_39/ProcessingFailureTest.java @@ -0,0 +1,194 @@ +/* + * HDFS Data Ingestion for PTH_06 use CFE-39 + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. + */ +package com.teragrep.cfe_39; + +import com.teragrep.cfe_39.consumers.kafka.DatabaseOutput; +import com.teragrep.cfe_39.consumers.kafka.RecordOffset; +import com.teragrep.cfe_39.metrics.DurationStatistics; +import com.teragrep.cfe_39.metrics.topic.TopicCounter; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FileUtil; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.function.Consumer; + +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; + +// Tests for processing of consumed kafka records with skipping of broken records disabled (both null and non rfc5424). +public class ProcessingFailureTest { + + private static final Logger LOGGER = LoggerFactory.getLogger(ProcessingFailureTest.class); + + private static MiniDFSCluster hdfsCluster; + private static File baseDir; + private static Config config; + private FileSystem fs; + + // Prepares known state for testing. + @BeforeEach + public void startMiniCluster() { + assertDoesNotThrow(() -> { + // Set system properties to use the valid configuration with skipping of broken records disabled. + System + .setProperty("cfe_39.config.location", System.getProperty("user.dir") + "/src/test/resources/failProcessing.application.properties"); + config = new Config(); + // Create a HDFS miniCluster + baseDir = Files.createTempDirectory("test_hdfs").toFile().getAbsoluteFile(); + hdfsCluster = new TestMiniClusterFactory().create(config, baseDir); + fs = new TestFileSystemFactory().create(config.getHdfsuri()); + }); + } + + // Teardown the minicluster + @AfterEach + public void teardownMiniCluster() { + assertDoesNotThrow(() -> { + fs.close(); + }); + hdfsCluster.shutdown(); + FileUtil.fullyDelete(baseDir); + } + + @Test + public void failNonRFC5424DatabaseOutputTest() { + // Initialize and register duration statistics + DurationStatistics durationStatistics = new DurationStatistics(); + durationStatistics.register(); + + // register per topic counting + List topicCounters = new CopyOnWriteArrayList<>(); + + assertDoesNotThrow(() -> { + + Consumer> output = new DatabaseOutput( + config, // Configuration settings + "topicName", // String, the name of the topic + durationStatistics, // RuntimeStatistics object from metrics + new TopicCounter("topicName") // TopicCounter object from metrics + ); + + ConsumerRecord record = new ConsumerRecord<>( + "topicName", + 0, + 1L, + "2022-04-25T07:34:50.806Z".getBytes(StandardCharsets.UTF_8), + "12>1 2022-04-25T07:34:50.806Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"c3f13f9a-05e2-41bd-b0ad-1eca6fd6fd9a\" source=\"source\" unixtime=\"1650872090806\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872090\"] [ERROR] 2022-04-25 07:34:50,806 com.teragrep.jla_02.Log4j Log - Log4j error says hi!" + .getBytes(StandardCharsets.UTF_8) + ); + RecordOffset recordOffsetObject = new RecordOffset( + record.topic(), + record.partition(), + record.offset(), + record.value() + ); + + List recordOffsetObjectList = new ArrayList<>(); + recordOffsetObjectList.add(recordOffsetObject); + Exception e = Assertions.assertThrows(Exception.class, () -> output.accept(recordOffsetObjectList)); + Assertions.assertEquals("com.teragrep.rlo_06.PriorityParseException: PRIORITY < missing", e.getMessage()); + Assertions.assertFalse(fs.exists(new Path(config.getHdfsPath() + "/" + "topicName" + "/" + "0.1"))); + // No files stored to hdfs. + }); + + } + + @Test + public void failNullRFC5424DatabaseOutputTest() { + // Initialize and register duration statistics + DurationStatistics durationStatistics = new DurationStatistics(); + durationStatistics.register(); + + // register per topic counting + List topicCounters = new CopyOnWriteArrayList<>(); + + assertDoesNotThrow(() -> { + + Consumer> output = new DatabaseOutput( + config, // Configuration settings + "topicName", // String, the name of the topic + durationStatistics, // RuntimeStatistics object from metrics + new TopicCounter("topicName") // TopicCounter object from metrics + ); + + ConsumerRecord record = new ConsumerRecord<>( + "topicName", + 0, + 1L, + "2022-04-25T07:34:50.806Z".getBytes(StandardCharsets.UTF_8), + null + ); + RecordOffset recordOffsetObject = new RecordOffset( + record.topic(), + record.partition(), + record.offset(), + record.value() + ); + + List recordOffsetObjectList = new ArrayList<>(); + recordOffsetObjectList.add(recordOffsetObject); + NullPointerException e = Assertions + .assertThrows(NullPointerException.class, () -> output.accept(recordOffsetObjectList)); + Assertions.assertEquals("Record with null content detected during processing.", e.getMessage()); + Assertions.assertFalse(fs.exists(new Path(config.getHdfsPath() + "/" + "topicName" + "/" + "0.1"))); + // No files stored to hdfs. + }); + + } +} diff --git a/src/test/java/com/teragrep/cfe_39/ProcessingTest.java b/src/test/java/com/teragrep/cfe_39/ProcessingTest.java new file mode 100644 index 00000000..17820c7f --- /dev/null +++ b/src/test/java/com/teragrep/cfe_39/ProcessingTest.java @@ -0,0 +1,311 @@ +/* + * HDFS Data Ingestion for PTH_06 use CFE-39 + * Copyright (C) 2021-2024 Suomen Kanuuna Oy + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as published by + * the Free Software Foundation, either version 3 of the License, or + * (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + * + * Additional permission under GNU Affero General Public License version 3 + * section 7 + * + * If you modify this Program, or any covered work, by linking or combining it + * with other code, such other code is not for that reason alone subject to any + * of the requirements of the GNU Affero GPL version 3 as long as this Program + * is the same Program as licensed from Suomen Kanuuna Oy without any additional + * modifications. + * + * Supplemented terms under GNU Affero General Public License version 3 + * section 7 + * + * Origin of the software must be attributed to Suomen Kanuuna Oy. Any modified + * versions must be marked as "Modified version of" The Program. + * + * Names of the licensors and authors may not be used for publicity purposes. + * + * No rights are granted for use of trade names, trademarks, or service marks + * which are in The Program if any. + * + * Licensee must indemnify licensors and authors for any liability that these + * contractual assumptions impose on licensors and authors. + * + * To the extent this program is licensed as part of the Commercial versions of + * Teragrep, the applicable Commercial License may apply to this file if you as + * a licensee so wish it. + */ +package com.teragrep.cfe_39; + +import com.teragrep.cfe_39.avro.SyslogRecord; +import com.teragrep.cfe_39.consumers.kafka.DatabaseOutput; +import com.teragrep.cfe_39.consumers.kafka.RecordOffset; +import com.teragrep.cfe_39.metrics.DurationStatistics; +import com.teragrep.cfe_39.metrics.topic.TopicCounter; +import org.apache.avro.file.DataFileStream; +import org.apache.avro.specific.SpecificDatumReader; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FileUtil; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.function.Consumer; + +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; + +// Tests for processing of consumed kafka records with skipping of broken records enabled (both null and non rfc5424). +public class ProcessingTest { + + private static final Logger LOGGER = LoggerFactory.getLogger(ProcessingTest.class); + + private static MiniDFSCluster hdfsCluster; + private static File baseDir; + private static Config config; + private FileSystem fs; + + // Prepares known state for testing. + @BeforeEach + public void startMiniCluster() { + assertDoesNotThrow(() -> { + // Set system properties to use the valid configuration. + System + .setProperty("cfe_39.config.location", System.getProperty("user.dir") + "/src/test/resources/valid.application.properties"); + config = new Config(); + // Create a HDFS miniCluster + baseDir = Files.createTempDirectory("test_hdfs").toFile().getAbsoluteFile(); + hdfsCluster = new TestMiniClusterFactory().create(config, baseDir); + fs = new TestFileSystemFactory().create(config.getHdfsuri()); + }); + } + + // Teardown the minicluster + @AfterEach + public void teardownMiniCluster() { + assertDoesNotThrow(() -> { + fs.close(); + }); + hdfsCluster.shutdown(); + FileUtil.fullyDelete(baseDir); + } + + @Test + public void skipNonRFC5424DatabaseOutputTest() { + // Initialize and register duration statistics + DurationStatistics durationStatistics = new DurationStatistics(); + durationStatistics.register(); + + // register per topic counting + List topicCounters = new CopyOnWriteArrayList<>(); + + assertDoesNotThrow(() -> { + + Consumer> output = new DatabaseOutput( + config, // Configuration settings + "topicName", // String, the name of the topic + durationStatistics, // RuntimeStatistics object from metrics + new TopicCounter("topicName") // TopicCounter object from metrics + ); + + ConsumerRecord record = new ConsumerRecord<>( + "topicName", + 0, + 1L, + "2022-04-25T07:34:50.806Z".getBytes(StandardCharsets.UTF_8), + "12>1 2022-04-25T07:34:50.806Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"c3f13f9a-05e2-41bd-b0ad-1eca6fd6fd9a\" source=\"source\" unixtime=\"1650872090806\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872090\"] [ERROR] 2022-04-25 07:34:50,806 com.teragrep.jla_02.Log4j Log - Log4j error says hi!" + .getBytes(StandardCharsets.UTF_8) + ); + RecordOffset recordOffsetObject = new RecordOffset( + record.topic(), + record.partition(), + record.offset(), + record.value() + ); + + List recordOffsetObjectList = new ArrayList<>(); + recordOffsetObjectList.add(recordOffsetObject); + output.accept(recordOffsetObjectList); + Assertions.assertEquals(1, fs.listStatus(new Path(config.getHdfsPath() + "/" + "topicName")).length); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "topicName" + "/" + "0.1"))); + // File in hdfs does not contain any records, but acts as a marker for kafka consumer offsets. + + // Assert that the file in hdfs contains the expected zero record. + + Path hdfsreadpath = new Path(config.getHdfsPath() + "/" + "topicName" + "/" + "0.1"); + //Init input stream + FSDataInputStream inputStream = fs.open(hdfsreadpath); + //The data is in AVRO-format, so it can't be read as a string. + DataFileStream reader = new DataFileStream<>( + inputStream, + new SpecificDatumReader<>(SyslogRecord.class) + ); + SyslogRecord syslogRecord = null; + LOGGER.info("\nReading records from file {}:", hdfsreadpath); + + Assertions.assertFalse(reader.hasNext()); + }); + + } + + @Test + public void skipNullRFC5424DatabaseOutputTest() { + // Initialize and register duration statistics + DurationStatistics durationStatistics = new DurationStatistics(); + durationStatistics.register(); + + // register per topic counting + List topicCounters = new CopyOnWriteArrayList<>(); + + assertDoesNotThrow(() -> { + + Consumer> output = new DatabaseOutput( + config, // Configuration settings + "topicName", // String, the name of the topic + durationStatistics, // RuntimeStatistics object from metrics + new TopicCounter("topicName") // TopicCounter object from metrics + ); + + ConsumerRecord record = new ConsumerRecord<>( + "topicName", + 0, + 1L, + "2022-04-25T07:34:50.806Z".getBytes(StandardCharsets.UTF_8), + null + ); + RecordOffset recordOffsetObject = new RecordOffset( + record.topic(), + record.partition(), + record.offset(), + record.value() + ); + + List recordOffsetObjectList = new ArrayList<>(); + recordOffsetObjectList.add(recordOffsetObject); + output.accept(recordOffsetObjectList); + Assertions.assertEquals(1, fs.listStatus(new Path(config.getHdfsPath() + "/" + "topicName")).length); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "topicName" + "/" + "0.1"))); + // File in hdfs does not contain any records, but acts as a marker for kafka consumer offsets. + + // Assert that the file in hdfs contains the expected zero record. + + Path hdfsreadpath = new Path(config.getHdfsPath() + "/" + "topicName" + "/" + "0.1"); + //Init input stream + FSDataInputStream inputStream = fs.open(hdfsreadpath); + //The data is in AVRO-format, so it can't be read as a string. + DataFileStream reader = new DataFileStream<>( + inputStream, + new SpecificDatumReader<>(SyslogRecord.class) + ); + SyslogRecord syslogRecord = null; + LOGGER.info("\nReading records from file {}:", hdfsreadpath); + + Assertions.assertFalse(reader.hasNext()); + }); + + } + + @Test + public void skipNullAndNonRFC5424DatabaseOutputTest() { + // Initialize and register duration statistics + DurationStatistics durationStatistics = new DurationStatistics(); + durationStatistics.register(); + + // register per topic counting + List topicCounters = new CopyOnWriteArrayList<>(); + + assertDoesNotThrow(() -> { + + Consumer> output = new DatabaseOutput( + config, // Configuration settings + "topicName", // String, the name of the topic + durationStatistics, // RuntimeStatistics object from metrics + new TopicCounter("topicName") // TopicCounter object from metrics + ); + + List recordOffsetObjectList = new ArrayList<>(); + + ConsumerRecord record = new ConsumerRecord<>( + "topicName", + 0, + 1L, + "2022-04-25T07:34:50.806Z".getBytes(StandardCharsets.UTF_8), + null + ); + RecordOffset recordOffsetObject = new RecordOffset( + record.topic(), + record.partition(), + record.offset(), + record.value() + ); + recordOffsetObjectList.add(recordOffsetObject); + + record = new ConsumerRecord<>( + "topicName", + 0, + 2L, + "2022-04-25T07:34:50.807Z".getBytes(StandardCharsets.UTF_8), + "12>1 2022-04-25T07:34:50.807Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"c3f13f9a-05e2-41bd-b0ad-1eca6fd6fd9a\" source=\"source\" unixtime=\"1650872090806\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872090\"] [ERROR] 2022-04-25 07:34:50,806 com.teragrep.jla_02.Log4j Log - Log4j error says hi!" + .getBytes(StandardCharsets.UTF_8) + ); + recordOffsetObject = new RecordOffset(record.topic(), record.partition(), record.offset(), record.value()); + recordOffsetObjectList.add(recordOffsetObject); + record = new ConsumerRecord<>( + "topicName", + 0, + 3L, + "2022-04-25T07:34:50.807Z".getBytes(StandardCharsets.UTF_8), + "<12>1 2022-04-25T07:34:50.807Z jla-02.default jla02logger - - [origin@48577 hostname=\"jla-02.default\"][event_id@48577 hostname=\"jla-02.default\" uuid=\"c3f13f9a-05e2-41bd-b0ad-1eca6fd6fd9a\" source=\"source\" unixtime=\"1650872090806\"][event_format@48577 original_format=\"rfc5424\"][event_node_relay@48577 hostname=\"cfe-06-0.cfe-06.default\" source=\"kafka-4.kafka.default.svc.cluster.local\" source_module=\"imrelp\"][event_version@48577 major=\"2\" minor=\"2\" hostname=\"cfe-06-0.cfe-06.default\" version_source=\"relay\"][event_node_router@48577 source=\"cfe-06-0.cfe-06.default.svc.cluster.local\" source_module=\"imrelp\" hostname=\"cfe-07-0.cfe-07.default\"][teragrep@48577 streamname=\"test:jla02logger:0\" directory=\"jla02logger\" unixtime=\"1650872090\"] [ERROR] 2022-04-25 07:34:50,806 com.teragrep.jla_02.Log4j Log - Log4j error says hi!" + .getBytes(StandardCharsets.UTF_8) + ); + recordOffsetObject = new RecordOffset(record.topic(), record.partition(), record.offset(), record.value()); + recordOffsetObjectList.add(recordOffsetObject); + output.accept(recordOffsetObjectList); + Assertions.assertEquals(1, fs.listStatus(new Path(config.getHdfsPath() + "/" + "topicName")).length); + Assertions.assertTrue(fs.exists(new Path(config.getHdfsPath() + "/" + "topicName" + "/" + "0.3"))); + + // Assert that the file in hdfs contains the expected single record. + + Path hdfsreadpath = new Path(config.getHdfsPath() + "/" + "topicName" + "/" + "0.3"); + //Init input stream + FSDataInputStream inputStream = fs.open(hdfsreadpath); + //The data is in AVRO-format, so it can't be read as a string. + DataFileStream reader = new DataFileStream<>( + inputStream, + new SpecificDatumReader<>(SyslogRecord.class) + ); + SyslogRecord syslogRecord = null; + LOGGER.info("\nReading records from file {}:", hdfsreadpath); + + Assertions.assertTrue(reader.hasNext()); + syslogRecord = reader.next(syslogRecord); + Assertions + .assertEquals( + "{\"timestamp\": 1650872090807000, \"directory\": \"jla02logger\", \"stream\": \"test:jla02logger:0\", \"host\": \"jla-02.default\", \"input\": \"imrelp:cfe-06-0.cfe-06.default:\", \"partition\": \"0\", \"offset\": 3, \"origin\": \"jla-02.default\", \"payload\": \"[ERROR] 2022-04-25 07:34:50,806 com.teragrep.jla_02.Log4j Log - Log4j error says hi!\"}", + syslogRecord.toString() + ); + Assertions.assertFalse(reader.hasNext()); + + }); + } +} diff --git a/src/test/resources/broken.application.properties b/src/test/resources/broken.application.properties index 2a6d929b..1ed21e27 100644 --- a/src/test/resources/broken.application.properties +++ b/src/test/resources/broken.application.properties @@ -24,6 +24,10 @@ consumer.max.poll.interval.ms=300000 consumer.useMockKafkaConsumer=true # The maximum file size for AVRO-files that are to be stored in HDFS database. maximumFileSize=3000 +# Boolean for deciding if records not in RFC5424 should be skipped or not. +skipNonRFC5424Records=true +# Boolean for deciding if empty RFC5424 records should be skipped or not. +skipEmptyRFC5424Records=true # HDFS pruning, use 157784760000 value while testing HDFS writes to ensure the test records are not pruned. 157784760000L pruneOffset=157784760000 # Kerberos @@ -34,4 +38,5 @@ hadoop.security.authorization=test dfs.namenode.kerberos.principal.pattern=test KerberosKeytabUser=test KerberosKeytabPath=test -dfs.client.use.datanode.hostname=false \ No newline at end of file +dfs.client.use.datanode.hostname=false +kerberosLoginAutorenewal=true \ No newline at end of file diff --git a/src/test/resources/failProcessing.application.properties b/src/test/resources/failProcessing.application.properties new file mode 100644 index 00000000..46219c46 --- /dev/null +++ b/src/test/resources/failProcessing.application.properties @@ -0,0 +1,44 @@ +# What topics are searched from kafka, regex +queueTopicPattern=^testConsumerTopic-*$ +# Number of consumers created to the consumer groups +numOfConsumers=2 +# Kafka bootstrap servers +consumer.bootstrap.servers=test +# Offset, should not be touched +consumer.auto.offset.reset=earliest +# Autocommit, should not be touched +consumer.enable.auto.commit=false +# Consumer group id, this is to track the progress of reading hte topic +consumer.group.id=cfe_39 +# Used security protocol and mechanism +consumer.security.protocol=SASL_PLAINTEXT +consumer.sasl.mechanism=PLAIN +# Maximum records per batch, note that too big number will cause massive load and can cause timeouts to trigger +consumer.max.poll.records=500 +# How much data can be fetched in one go +consumer.fetch.max.bytes=1073741820 +# How long for request before timing out. Note that too big max poll records size can cause this to trigger +consumer.request.timeout.ms=300000 +consumer.max.poll.interval.ms=300000 +# For testing only, remove for prod. +consumer.useMockKafkaConsumer=true +# The maximum file size for AVRO-files that are to be stored in HDFS database. +maximumFileSize=3000 +# Boolean for deciding if records not in RFC5424 should be skipped or not. +skipNonRFC5424Records=false +# Boolean for deciding if empty RFC5424 records should be skipped or not. +skipEmptyRFC5424Records=false +# HDFS pruning, use 157784760000 value while testing HDFS writes to ensure the test records are not pruned. 157784760000L +pruneOffset=157784760000 +# HDFS uri +hdfsuri=hdfs://localhost:45937/ +# Kerberos +java.security.krb5.kdc=test +java.security.krb5.realm=test +hadoop.security.authentication=test +hadoop.security.authorization=test +dfs.namenode.kerberos.principal.pattern=test +KerberosKeytabUser=test +KerberosKeytabPath=test +dfs.client.use.datanode.hostname=false +kerberosLoginAutorenewal=true \ No newline at end of file diff --git a/src/test/resources/valid.application.properties b/src/test/resources/valid.application.properties index 2f03cba0..1b9a22df 100644 --- a/src/test/resources/valid.application.properties +++ b/src/test/resources/valid.application.properties @@ -24,6 +24,10 @@ consumer.max.poll.interval.ms=300000 consumer.useMockKafkaConsumer=true # The maximum file size for AVRO-files that are to be stored in HDFS database. maximumFileSize=3000 +# Boolean for deciding if records not in RFC5424 should be skipped or not. +skipNonRFC5424Records=true +# Boolean for deciding if empty RFC5424 records should be skipped or not. +skipEmptyRFC5424Records=true # HDFS pruning, use 157784760000 value while testing HDFS writes to ensure the test records are not pruned. 157784760000L pruneOffset=157784760000 # HDFS uri @@ -36,4 +40,5 @@ hadoop.security.authorization=test dfs.namenode.kerberos.principal.pattern=test KerberosKeytabUser=test KerberosKeytabPath=test -dfs.client.use.datanode.hostname=false \ No newline at end of file +dfs.client.use.datanode.hostname=false +kerberosLoginAutorenewal=true \ No newline at end of file From 18de5a1aa3be72567ccec11d837b0525ac926ba5 Mon Sep 17 00:00:00 2001 From: Mikko Kortelainen Date: Tue, 20 Aug 2024 15:18:12 +0300 Subject: [PATCH 145/146] apply spotless (#38) --- src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java | 1 - 1 file changed, 1 deletion(-) diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java index ab5c9632..d237de23 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java @@ -50,7 +50,6 @@ import org.apache.hadoop.fs.*; import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.HdfsConfiguration; -import org.apache.hadoop.security.UserGroupInformation; import org.slf4j.Logger; import org.slf4j.LoggerFactory; From 220d0d5f38ff41d20ddbe37b751bce5ed84da451 Mon Sep 17 00:00:00 2001 From: Tiihott <119838215+Tiihott@users.noreply.github.com> Date: Tue, 20 Aug 2024 16:40:59 +0300 Subject: [PATCH 146/146] Added encryption flags for configuration (#40) * Added encryption flags for configuration * Added missing flags to example/test configuration --- rpm/resources/application.properties | 4 +++- src/main/java/com/teragrep/cfe_39/Config.java | 19 +++++++++++++++++++ .../cfe_39/consumers/kafka/HDFSWrite.java | 4 ++++ .../consumers/kafka/HdfsDataIngestion.java | 3 +++ .../resources/broken.application.properties | 4 +++- .../failProcessing.application.properties | 4 +++- .../resources/valid.application.properties | 4 +++- 7 files changed, 38 insertions(+), 4 deletions(-) diff --git a/rpm/resources/application.properties b/rpm/resources/application.properties index 06ab6c7c..fc5100cb 100644 --- a/rpm/resources/application.properties +++ b/rpm/resources/application.properties @@ -47,4 +47,6 @@ dfs.namenode.kerberos.principal.pattern=test KerberosKeytabUser=test KerberosKeytabPath=test dfs.client.use.datanode.hostname=false -kerberosLoginAutorenewal=true \ No newline at end of file +kerberosLoginAutorenewal=true +dfs.data.transfer.protection=test +dfs.encrypt.data.transfer.cipher.suites=test \ No newline at end of file diff --git a/src/main/java/com/teragrep/cfe_39/Config.java b/src/main/java/com/teragrep/cfe_39/Config.java index 24d43b3d..c29e1ed9 100644 --- a/src/main/java/com/teragrep/cfe_39/Config.java +++ b/src/main/java/com/teragrep/cfe_39/Config.java @@ -80,6 +80,8 @@ public class Config { private final long pruneOffset; private final boolean skipNonRFC5424Records; private final boolean skipEmptyRFC5424Records; + private final String dfsDataTransferProtection; + private final String dfsEncryptDataTransferCipherSuites; public Config() throws IOException { Properties properties = new Properties(); @@ -147,6 +149,15 @@ public Config() throws IOException { } this.kerberosTestMode = properties.getProperty("dfs.client.use.datanode.hostname", "false"); + this.dfsDataTransferProtection = properties.getProperty("dfs.data.transfer.protection"); + if (this.dfsDataTransferProtection == null) { + throw new IllegalArgumentException("dfsDataTransferProtection not set"); + } + this.dfsEncryptDataTransferCipherSuites = properties.getProperty("dfs.encrypt.data.transfer.cipher.suites"); + if (this.dfsEncryptDataTransferCipherSuites == null) { + throw new IllegalArgumentException("dfsEncryptDataTransferCipherSuites not set"); + } + // kafka this.queueTopicPattern = properties.getProperty("queueTopicPattern", "^.*$"); this.numOfConsumers = Integer.parseInt(properties.getProperty("numOfConsumers", "1")); @@ -274,4 +285,12 @@ public boolean getSkipEmptyRFC5424Records() { public String getKerberosLoginAutorenewal() { return kerberosLoginAutorenewal; } + + public String getDfsDataTransferProtection() { + return dfsDataTransferProtection; + } + + public String getDfsEncryptDataTransferCipherSuites() { + return dfsEncryptDataTransferCipherSuites; + } } diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java index d237de23..c949ee81 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HDFSWrite.java @@ -131,6 +131,10 @@ These values should be fetched from config and other input parameters (topic+par // server principal, the kerberos principle that the namenode is using conf.set("dfs.namenode.kerberos.principal.pattern", config.getKerberosPrincipal()); + // set sasl + conf.set("dfs.data.transfer.protection", config.getDfsDataTransferProtection()); + conf.set("dfs.encrypt.data.transfer.cipher.suites", config.getDfsEncryptDataTransferCipherSuites()); + // filesystem for HDFS access is set here fs = FileSystem.get(conf); } diff --git a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HdfsDataIngestion.java b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HdfsDataIngestion.java index d710a8dd..305035bd 100644 --- a/src/main/java/com/teragrep/cfe_39/consumers/kafka/HdfsDataIngestion.java +++ b/src/main/java/com/teragrep/cfe_39/consumers/kafka/HdfsDataIngestion.java @@ -133,6 +133,9 @@ public HdfsDataIngestion(Config config) throws IOException { /* server principal the kerberos principle that the namenode is using*/ conf.set("dfs.namenode.kerberos.principal.pattern", config.getKerberosPrincipal()); + // set sasl + conf.set("dfs.data.transfer.protection", config.getDfsDataTransferProtection()); + conf.set("dfs.encrypt.data.transfer.cipher.suites", config.getDfsEncryptDataTransferCipherSuites()); // set usergroup stuff UserGroupInformation.setConfiguration(conf); UserGroupInformation.loginUserFromKeytab(config.getKerberosKeytabUser(), config.getKerberosKeytabPath()); diff --git a/src/test/resources/broken.application.properties b/src/test/resources/broken.application.properties index 1ed21e27..433eee07 100644 --- a/src/test/resources/broken.application.properties +++ b/src/test/resources/broken.application.properties @@ -39,4 +39,6 @@ dfs.namenode.kerberos.principal.pattern=test KerberosKeytabUser=test KerberosKeytabPath=test dfs.client.use.datanode.hostname=false -kerberosLoginAutorenewal=true \ No newline at end of file +kerberosLoginAutorenewal=true +dfs.data.transfer.protection=test +dfs.encrypt.data.transfer.cipher.suites=test \ No newline at end of file diff --git a/src/test/resources/failProcessing.application.properties b/src/test/resources/failProcessing.application.properties index 46219c46..55bc98d2 100644 --- a/src/test/resources/failProcessing.application.properties +++ b/src/test/resources/failProcessing.application.properties @@ -41,4 +41,6 @@ dfs.namenode.kerberos.principal.pattern=test KerberosKeytabUser=test KerberosKeytabPath=test dfs.client.use.datanode.hostname=false -kerberosLoginAutorenewal=true \ No newline at end of file +kerberosLoginAutorenewal=true +dfs.data.transfer.protection=test +dfs.encrypt.data.transfer.cipher.suites=test \ No newline at end of file diff --git a/src/test/resources/valid.application.properties b/src/test/resources/valid.application.properties index 1b9a22df..acbcf93d 100644 --- a/src/test/resources/valid.application.properties +++ b/src/test/resources/valid.application.properties @@ -41,4 +41,6 @@ dfs.namenode.kerberos.principal.pattern=test KerberosKeytabUser=test KerberosKeytabPath=test dfs.client.use.datanode.hostname=false -kerberosLoginAutorenewal=true \ No newline at end of file +kerberosLoginAutorenewal=true +dfs.data.transfer.protection=test +dfs.encrypt.data.transfer.cipher.suites=test \ No newline at end of file