From a5699777f928fae285a3b3ed88aed1451527e88d Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Wed, 11 Sep 2024 15:48:27 +0300 Subject: [PATCH 01/62] Migration of the branch to github, InstantiationTest passed successfully. --- pom.xml | 54 +++++ .../pth_06/ArchiveMicroStreamReader.java | 156 ++++++++++-- .../HdfsTopicPartitionOffsetMetadata.java | 31 +++ .../com/teragrep/pth_06/config/Config.java | 11 +- .../teragrep/pth_06/config/HdfsConfig.java | 40 ++++ .../teragrep/pth_06/planner/HdfsDBClient.java | 142 +++++++++++ .../teragrep/pth_06/planner/HdfsQuery.java | 20 ++ .../pth_06/planner/HdfsQueryProcessor.java | 163 +++++++++++++ .../teragrep/pth_06/planner/KafkaQuery.java | 5 + .../pth_06/planner/KafkaQueryProcessor.java | 36 +++ .../planner/offset/DatasourceOffset.java | 20 ++ .../pth_06/planner/offset/HdfsOffset.java | 63 +++++ .../offset/SerializedDatasourceOffset.java | 41 +++- .../planner/walker/HdfsConditionWalker.java | 50 ++++ .../com/teragrep/pth_06/scheduler/Batch.java | 17 +- .../teragrep/pth_06/scheduler/BatchSlice.java | 23 +- .../scheduler/HdfsBatchSliceCollection.java | 71 ++++++ .../task/HdfsMicroBatchInputPartition.java | 4 + .../HdfsMicroBatchInputPartitionReader.java | 4 + .../teragrep/pth_06/task/hdfs/AvroReader.java | 4 + .../pth_06/task/hdfs/HdfsRecordConverter.java | 4 + .../teragrep/pth_06/InstantiationTest.java | 21 +- .../pth_06/MockTeragrepDatasource.java | 9 +- .../com/teragrep/pth_06/mockHdfsFiles/0.13 | Bin 0 -> 1982 bytes .../com/teragrep/pth_06/mockHdfsFiles/0.8 | Bin 0 -> 2692 bytes .../com/teragrep/pth_06/planner/MockHDFS.java | 223 ++++++++++++++++++ 26 files changed, 1181 insertions(+), 31 deletions(-) create mode 100644 src/main/java/com/teragrep/pth_06/HdfsTopicPartitionOffsetMetadata.java create mode 100644 src/main/java/com/teragrep/pth_06/config/HdfsConfig.java create mode 100644 src/main/java/com/teragrep/pth_06/planner/HdfsDBClient.java create mode 100644 src/main/java/com/teragrep/pth_06/planner/HdfsQuery.java create mode 100644 src/main/java/com/teragrep/pth_06/planner/HdfsQueryProcessor.java create mode 100644 src/main/java/com/teragrep/pth_06/planner/offset/HdfsOffset.java create mode 100644 src/main/java/com/teragrep/pth_06/planner/walker/HdfsConditionWalker.java create mode 100644 src/main/java/com/teragrep/pth_06/scheduler/HdfsBatchSliceCollection.java create mode 100644 src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartition.java create mode 100644 src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReader.java create mode 100644 src/main/java/com/teragrep/pth_06/task/hdfs/AvroReader.java create mode 100644 src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConverter.java create mode 100644 src/test/java/com/teragrep/pth_06/mockHdfsFiles/0.13 create mode 100644 src/test/java/com/teragrep/pth_06/mockHdfsFiles/0.8 create mode 100644 src/test/java/com/teragrep/pth_06/planner/MockHDFS.java diff --git a/pom.xml b/pom.xml index 47ffaa4a..d9862b0e 100644 --- a/pom.xml +++ b/pom.xml @@ -31,6 +31,7 @@ -SNAPSHOT 1.8 2.3.0 + 3.3.6 1.8 1.8 UTF-8 @@ -104,18 +105,36 @@ spark-streaming_2.12 3.4.0 provided + + + org.apache.hadoop + * + + org.apache.spark spark-sql-kafka-0-10_2.12 3.4.0 provided + + + org.apache.hadoop + * + + org.apache.spark spark-sql_2.12 3.4.0 provided + + + org.apache.hadoop + * + + com.amazonaws @@ -130,6 +149,41 @@ config 1.4.0 + + + org.apache.hadoop + hadoop-client + ${hadoop.version} + + + org.apache.hadoop + hadoop-minicluster + ${hadoop.version} + test + + + com.google.code.gson + gson + + + + + org.apache.hadoop + hadoop-common + ${hadoop.version} + + + com.google.code.gson + gson + + + + + org.apache.hadoop + hadoop-hdfs + ${hadoop.version} + test + org.mockito diff --git a/src/main/java/com/teragrep/pth_06/ArchiveMicroStreamReader.java b/src/main/java/com/teragrep/pth_06/ArchiveMicroStreamReader.java index d8025bc8..c3d2bcbb 100644 --- a/src/main/java/com/teragrep/pth_06/ArchiveMicroStreamReader.java +++ b/src/main/java/com/teragrep/pth_06/ArchiveMicroStreamReader.java @@ -46,9 +46,11 @@ package com.teragrep.pth_06; import com.google.common.annotations.VisibleForTesting; +import com.google.gson.JsonArray; import com.teragrep.pth_06.config.Config; import com.teragrep.pth_06.planner.*; import com.teragrep.pth_06.planner.offset.DatasourceOffset; +import com.teragrep.pth_06.planner.offset.HdfsOffset; import com.teragrep.pth_06.planner.offset.KafkaOffset; import com.teragrep.pth_06.scheduler.*; import com.teragrep.pth_06.task.ArchiveMicroBatchInputPartition; @@ -87,6 +89,8 @@ public final class ArchiveMicroStreamReader implements MicroBatchStream { private final Config config; private final ArchiveQuery aq; private final KafkaQuery kq; + private final HdfsQuery hq; + private final JsonArray hdfsOffsets; /** * Constructor for ArchiveMicroStreamReader @@ -98,6 +102,14 @@ public final class ArchiveMicroStreamReader implements MicroBatchStream { this.config = config; + if (config.isHdfsEnabled) { + this.hq = new HdfsQueryProcessor(config); + hdfsOffsets = hq.hdfsOffsetMapToJSON(); + } else { + this.hq = null; + hdfsOffsets = null; + } + if (config.isArchiveEnabled) { this.aq = new ArchiveQueryProcessor(config); } @@ -106,7 +118,12 @@ public final class ArchiveMicroStreamReader implements MicroBatchStream { } if (config.isKafkaEnabled) { - this.kq = new KafkaQueryProcessor(config); + if (config.isHdfsEnabled) { + this.kq = new KafkaQueryProcessor(config); + this.kq.seekToHdfsOffsets(hdfsOffsets); + }else { + this.kq = new KafkaQueryProcessor(config); + } } else { this.kq = null; @@ -119,10 +136,19 @@ public final class ArchiveMicroStreamReader implements MicroBatchStream { * Used for testing. */ @VisibleForTesting - ArchiveMicroStreamReader(ArchiveQuery aq, KafkaQuery kq, Config config) { + ArchiveMicroStreamReader(HdfsQuery hq, ArchiveQuery aq, KafkaQuery kq, Config config) { this.config = config; - this.aq = aq; - this.kq = kq; + this.aq = aq; // Uses its own hardcoded query string defined in MockTeragrepDatasource. + this.kq = kq; // Skips using query string (and thus topic filtering) altogether. + this.hq = hq; // Uses the query string from config for topic filtering. + if (this.hq != null && this.kq != null) { + hdfsOffsets = this.hq.hdfsOffsetMapToJSON(); + this.kq.seekToHdfsOffsets(hdfsOffsets); + } else { + hdfsOffsets = null; + } + + LOGGER.debug("@VisibleForTesting MicroBatchReader> initialized"); } @@ -135,25 +161,68 @@ public final class ArchiveMicroStreamReader implements MicroBatchStream { */ @Override public Offset initialOffset() { + // After rebase is complete: Refactor the DatasourceOffset and SerializedDatasourceOffset if the 8x else-if statements are too much. + // archive only: subtract 3600s (1 hour) from earliest to return first row (start exclusive) DatasourceOffset rv; - if (this.config.isArchiveEnabled && !this.config.isKafkaEnabled) { + if (this.config.isArchiveEnabled && !this.config.isKafkaEnabled && !this.config.isHdfsEnabled) { // only archive rv = new DatasourceOffset(new LongOffset(this.aq.getInitialOffset() - 3600L)); } - else if (!this.config.isArchiveEnabled && this.config.isKafkaEnabled) { + else if (!this.config.isArchiveEnabled && this.config.isKafkaEnabled && !this.config.isHdfsEnabled) { // only kafka rv = new DatasourceOffset(new KafkaOffset(this.kq.getBeginningOffsets(null))); } - else if (this.config.isArchiveEnabled) { - // both - rv = new DatasourceOffset( + else if (!this.config.isArchiveEnabled && !this.config.isKafkaEnabled && this.config.isHdfsEnabled) { + // only HDFS + rv = new DatasourceOffset(new HdfsOffset(this.hq.getBeginningOffsets().getOffsetMap())); + } + else if (this.config.isArchiveEnabled && this.config.isKafkaEnabled && !this.config.isHdfsEnabled) { + // kafka and archive + rv = new DatasourceOffset( new LongOffset(this.aq.getInitialOffset() - 3600L), new KafkaOffset(this.kq.getBeginningOffsets(null)) ); } + else if (this.config.isArchiveEnabled && !this.config.isKafkaEnabled && this.config.isHdfsEnabled) { + // archive and HDFS + rv = new DatasourceOffset( + new HdfsOffset(this.hq.getBeginningOffsets().getOffsetMap()), + new LongOffset(this.aq.getInitialOffset() - 3600L) + ); + } + else if (!this.config.isArchiveEnabled && this.config.isKafkaEnabled && this.config.isHdfsEnabled) { + // Kafka and HDFS, check if any files are available from HDFS. + if (hdfsOffsets.size() > 0) { + rv = new DatasourceOffset( + new HdfsOffset(this.hq.getBeginningOffsets().getOffsetMap()), + new KafkaOffset(this.kq.getConsumerPositions(hdfsOffsets)) + ); + } else { + rv = new DatasourceOffset( + new HdfsOffset(this.hq.getBeginningOffsets().getOffsetMap()), + new KafkaOffset(this.kq.getBeginningOffsets(null)) + ); + } + } + else if (this.config.isArchiveEnabled && this.config.isKafkaEnabled && this.config.isHdfsEnabled) { + // all three, check if any files are available from HDFS. + if (hdfsOffsets.size() > 0) { + rv = new DatasourceOffset( + new HdfsOffset(this.hq.getBeginningOffsets().getOffsetMap()), + new LongOffset(this.aq.getInitialOffset() - 3600L), + new KafkaOffset(this.kq.getConsumerPositions(hdfsOffsets)) + ); + }else { + rv = new DatasourceOffset( + new HdfsOffset(this.hq.getBeginningOffsets().getOffsetMap()), + new LongOffset(this.aq.getInitialOffset() - 3600L), + new KafkaOffset(this.kq.getBeginningOffsets(null)) + ); + } + } else { - // neither + // none throw new IllegalStateException("no datasources enabled, can't get initial offset"); } LOGGER.debug("offset[initial]= {}", rv); @@ -172,6 +241,9 @@ public void commit(Offset offset) { if (this.config.isArchiveEnabled) { this.aq.commit(((DatasourceOffset) offset).getArchiveOffset().offset()); } + if (this.config.isHdfsEnabled) { + this.hq.commit(((DatasourceOffset)offset).getHdfsOffset()); + } } /** {@inheritDoc} */ @@ -187,24 +259,46 @@ public void stop() { */ @Override public Offset latestOffset() { + // After rebase is complete: Refactor the DatasourceOffset and SerializedDatasourceOffset if the 8x else-if statements are too much. + DatasourceOffset rv; - if (this.config.isArchiveEnabled && !this.config.isKafkaEnabled) { + if (this.config.isArchiveEnabled && !this.config.isKafkaEnabled && !this.config.isHdfsEnabled) { // only archive rv = new DatasourceOffset(new LongOffset(this.aq.incrementAndGetLatestOffset())); } - else if (!this.config.isArchiveEnabled && this.config.isKafkaEnabled) { + else if (!this.config.isArchiveEnabled && this.config.isKafkaEnabled && !this.config.isHdfsEnabled) { // only kafka rv = new DatasourceOffset(new KafkaOffset(this.kq.getInitialEndOffsets())); - } - else if (this.config.isArchiveEnabled) { - // both - rv = new DatasourceOffset( + } else if (!this.config.isArchiveEnabled && !this.config.isKafkaEnabled && this.config.isHdfsEnabled) { + // only hdfs + rv = new DatasourceOffset(new HdfsOffset(this.hq.getInitialEndOffsets().getOffsetMap())); + } else if (this.config.isArchiveEnabled && this.config.isKafkaEnabled && !this.config.isHdfsEnabled) { + // kafka and archive + rv = new DatasourceOffset( new LongOffset(this.aq.incrementAndGetLatestOffset()), new KafkaOffset(this.kq.getInitialEndOffsets()) ); - } - else { - // neither + } else if (this.config.isArchiveEnabled && !this.config.isKafkaEnabled && this.config.isHdfsEnabled) { + // archive and hdfs + rv = new DatasourceOffset( + new HdfsOffset(this.hq.getInitialEndOffsets().getOffsetMap()), + new LongOffset(this.aq.incrementAndGetLatestOffset()) + ); + } else if (!this.config.isArchiveEnabled && this.config.isKafkaEnabled && this.config.isHdfsEnabled) { + // Kafka and HDFS + rv = new DatasourceOffset( + new HdfsOffset(this.hq.getInitialEndOffsets().getOffsetMap()), + new KafkaOffset(this.kq.getInitialEndOffsets()) + ); + } else if (this.config.isArchiveEnabled && this.config.isKafkaEnabled && this.config.isHdfsEnabled) { + // all three + rv = new DatasourceOffset( + new HdfsOffset(this.hq.getInitialEndOffsets().getOffsetMap()), + new LongOffset(this.aq.incrementAndGetLatestOffset()), + new KafkaOffset(this.kq.getInitialEndOffsets()) + ); + } else { + // none throw new IllegalStateException("no datasources enabled, can't get latest offset"); } @@ -223,7 +317,7 @@ else if (this.config.isArchiveEnabled) { public InputPartition[] planInputPartitions(Offset start, Offset end) { List inputPartitions = new LinkedList<>(); - Batch currentBatch = new Batch(config, aq, kq).processRange(start, end); + Batch currentBatch = new Batch(config, hq, aq, kq).processRange(start, end); for (LinkedList taskObjectList : currentBatch) { @@ -252,6 +346,28 @@ public InputPartition[] planInputPartitions(Offset start, Offset end) { ); } + // HDFS tasks + LinkedList hdfsTaskList = new LinkedList<>(); + for (BatchSlice batchSlice : taskObjectList) { + if (batchSlice.type.equals(BatchSlice.Type.HDFS)) { + hdfsTaskList.add(batchSlice.hdfsTopicPartitionOffsetMetadata); + } + } + + if (!hdfsTaskList.isEmpty()) { + // BatchSliceType.HDFS contains the metadata for the HDFS files that contain the records that are being queried. Available topics in HDFS are already filtered based on the spark query conditions. + // The records that are inside the files are fetched and processed in the tasker. Tasker does rest of the filtering based on the given query conditions, for example the cutoff epoch handling between the records that are fetched from S3 and HDFS. + // The Spark planner/scheduler is only single-threaded while tasker is multithreaded. Planner is not suitable for fetching and processing all the records, it should be done in tasker which will handle the processing in multithreaded environment based on batch slices. + + // Implement HdfsMicroBatchInputPartition usage here. If needed change the slice creation to be incremental like it is with archive, for that create HDFS variant of the incrementAndGetLatestOffset()-method from ArchiveQueryProcessor. + // At the moment it fetches all the metadata available at once and puts them into hdfsTaskList. + /*inputPartitions.add(new HdfsMicroBatchInputPartition( + config.hdfsConfig, + hdfsTaskList + ));*/ + } + + // kafka tasks for (BatchSlice batchSlice : taskObjectList) { if (batchSlice.type.equals(BatchSlice.Type.KAFKA)) { diff --git a/src/main/java/com/teragrep/pth_06/HdfsTopicPartitionOffsetMetadata.java b/src/main/java/com/teragrep/pth_06/HdfsTopicPartitionOffsetMetadata.java new file mode 100644 index 00000000..242ed316 --- /dev/null +++ b/src/main/java/com/teragrep/pth_06/HdfsTopicPartitionOffsetMetadata.java @@ -0,0 +1,31 @@ +package com.teragrep.pth_06; + +import org.apache.kafka.common.TopicPartition; + +import java.io.Serializable; + +// Class for holding serializable metadata of HDFS files containing kafka records. +// Maybe change the class name to something more appropriate. ie. HdfsFileMetadata +public class HdfsTopicPartitionOffsetMetadata implements Serializable { + public final TopicPartition topicPartition; // Represents the Kafka topic partition which records the file contains. + public final long endOffset; // Represents the offset of the record that was last added to the file. + public final String hdfsFilePath; // Represents the file path where the file resides in HDFS. + public final long hdfsFileSize; // Represents the size of the file in HDFS. Used for scheduling the batch slice. + + public HdfsTopicPartitionOffsetMetadata(TopicPartition topicPartition, long offset, String filePath, long fileSize) { + this.topicPartition = topicPartition; + this.endOffset = offset; + this.hdfsFilePath = filePath; + this.hdfsFileSize = fileSize; + } + + @Override + public String toString() { + return "HdfsTopicPartitionOffsetMetadata{" + + "topicPartition=" + topicPartition + + ", endOffset=" + endOffset + + ", hdfsFilePath=" + hdfsFilePath + + ", hdfsFileSize=" + hdfsFileSize + + '}'; + } +} \ No newline at end of file diff --git a/src/main/java/com/teragrep/pth_06/config/Config.java b/src/main/java/com/teragrep/pth_06/config/Config.java index 5da4d739..d2659891 100644 --- a/src/main/java/com/teragrep/pth_06/config/Config.java +++ b/src/main/java/com/teragrep/pth_06/config/Config.java @@ -59,12 +59,14 @@ public final class Config { public final String query; public final ArchiveConfig archiveConfig; public final KafkaConfig kafkaConfig; + public final HdfsConfig hdfsConfig; public final BatchConfig batchConfig; public final AuditConfig auditConfig; public final boolean isArchiveEnabled; public final boolean isKafkaEnabled; + public final boolean isHdfsEnabled; public final boolean isMetadataQuery; @@ -93,8 +95,15 @@ public Config(Map opts) { kafkaConfig = new KafkaConfig(); } + isHdfsEnabled = opts.getOrDefault("hdfs.enabled", "false").equalsIgnoreCase("true"); + if (isHdfsEnabled) { + hdfsConfig = new HdfsConfig(opts); + } else { + hdfsConfig = new HdfsConfig(); + } + // check that at least one datasource is enabled - if (!isArchiveEnabled && !isKafkaEnabled) { + if (!isArchiveEnabled && !isKafkaEnabled && !isHdfsEnabled) { throw new IllegalStateException("No datasources enabled"); } diff --git a/src/main/java/com/teragrep/pth_06/config/HdfsConfig.java b/src/main/java/com/teragrep/pth_06/config/HdfsConfig.java new file mode 100644 index 00000000..6b305f65 --- /dev/null +++ b/src/main/java/com/teragrep/pth_06/config/HdfsConfig.java @@ -0,0 +1,40 @@ +package com.teragrep.pth_06.config; + +import java.time.Instant; +import java.util.Map; + +public final class HdfsConfig { + + // HDFS + public final long cutoffOffset; // Represents the cutoff epoch which dictates which files should not be fetched from HDFS based on their timestamps. + public final String hdfsPath; // Represents the working directory path in HDFS filesystem. + public final String hdfsUri; // Represents the address of the HDFS server. + public final boolean useMockHdfsDatabase; // Represents the configuration parameter if mock database should be used or not. Used only for testing. + + public final boolean isStub; + public HdfsConfig(Map opts) { + cutoffOffset = Long.parseLong(opts.getOrDefault("hdfs.hdfsCutoffOffset", String.valueOf(Instant.now().toEpochMilli() - 72*3600000))); // Default is -72h from now + hdfsPath = getOrThrow(opts, "hdfs.hdfsPath"); + hdfsUri = getOrThrow(opts, "hdfs.hdfsUri"); + String useMockHdfsDatabaseString = opts.getOrDefault("useMockHdfsDatabase", "false"); + useMockHdfsDatabase = "true".equals(useMockHdfsDatabaseString); + isStub = false; + } + + public HdfsConfig() { + cutoffOffset = 0L; + hdfsPath = ""; + hdfsUri = ""; + useMockHdfsDatabase = false; + isStub = true; + } + + private String getOrThrow(Map opts, String key) { + String value = opts.get(key); + if (value == null) { + throw new IllegalArgumentException("Configuration item '" + key + "' was null!"); + } + return value; + } + +} \ No newline at end of file diff --git a/src/main/java/com/teragrep/pth_06/planner/HdfsDBClient.java b/src/main/java/com/teragrep/pth_06/planner/HdfsDBClient.java new file mode 100644 index 00000000..9fa97d33 --- /dev/null +++ b/src/main/java/com/teragrep/pth_06/planner/HdfsDBClient.java @@ -0,0 +1,142 @@ +package com.teragrep.pth_06.planner; + +import com.teragrep.pth_06.config.Config; +import com.teragrep.pth_06.HdfsTopicPartitionOffsetMetadata; +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.LinkedList; + +// Searches the HDFS database for files contained in directories based on topicsRegexString. Creates a list of objects where each object contains the metadata of a single HDFS-file. +public class HdfsDBClient { + + private final Logger LOGGER = LoggerFactory.getLogger(HdfsDBClient.class); + private final String path; + private final FileSystem fs; + private final Configuration conf; + private final String hdfsuri; + private final String topicsRegexString; + private final long ignoreBeforeEpoch; + + public HdfsDBClient(Config config, String topicsRegexString) throws IOException { + this.topicsRegexString = topicsRegexString; + this.ignoreBeforeEpoch = config.hdfsConfig.cutoffOffset; // Defines the minimum time window / cutoff epoch for making sure that the files which metadata is fetched by the planner are not mistakenly deleted by cfe_39 pruning before tasker can process the records inside. + // Implement HDFS FileSystem access here + if (config.hdfsConfig.useMockHdfsDatabase) { + // Code for initializing the class in test mode without kerberos. + hdfsuri = config.hdfsConfig.hdfsUri; // 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. + // These values should be fetched from config and other input parameters (topic+partition+offset). + path = config.hdfsConfig.hdfsPath; // Example path: hdfs://localhost:34509/opt/teragrep/cfe_39/srv/ + + // ====== 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.hdfsConfig.hdfsUri; // 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.hdfsConfig.hdfsPath; + + + // 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? + + // TODO: Add all the Kerberos parameters to Config + + // 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); + } + } + + // this queries and pulls the distinct file metadata values to the partitionList according to the given query conditions (condition only applies to topic names in planner side). + public LinkedList pullToPartitionList() throws IOException { + LinkedList rv = new LinkedList<>(); + // path holds the fileSystem path to the directory that holds a collection of other directories, each different directory representing a different topic. + FileStatus[] directoryStatuses = fs.listStatus(new Path(path), topicFilter); + // If the path only holds one directory, fileStatuses will only hold one FileStatus object which returns this value when fileStatus.getPath() is called: + // hdfs://localhost:34509/opt/teragrep/cfe_39/srv/testConsumerTopic + + // 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]; + HdfsTopicPartitionOffsetMetadata temp = new HdfsTopicPartitionOffsetMetadata(new TopicPartition(topic, Integer.parseInt(partition)), Integer.parseInt(offset), r2.getPath().toString(), r2.getLen()); + // Add the HdfsTopicPartitionOffsetMetadata object to the rv only if the file's modification timestamp is above ignoreBeforeEpoch. Timestamps are in milliseconds. + if (r2.getModificationTime() >= ignoreBeforeEpoch) { + rv.add(temp); + } + } + } + }else { + LOGGER.info("No matching directories found"); + } + return rv; + } + private final PathFilter topicFilter = new PathFilter() { + @Override + public boolean accept(Path path) { + return path.getName().matches(topicsRegexString); // Catches the directory names. + } + }; + +} \ No newline at end of file diff --git a/src/main/java/com/teragrep/pth_06/planner/HdfsQuery.java b/src/main/java/com/teragrep/pth_06/planner/HdfsQuery.java new file mode 100644 index 00000000..3ba15afa --- /dev/null +++ b/src/main/java/com/teragrep/pth_06/planner/HdfsQuery.java @@ -0,0 +1,20 @@ +package com.teragrep.pth_06.planner; + +import com.google.gson.JsonArray; +import com.teragrep.pth_06.HdfsTopicPartitionOffsetMetadata; +import com.teragrep.pth_06.planner.offset.HdfsOffset; +import org.apache.kafka.common.TopicPartition; + +import java.util.LinkedList; +import java.util.Map; + +// TODO: Make an interface for for HDFS query. methods are placeholders at the moment. + +// TODO: Fetches semi-recent data from HDFS. +public interface HdfsQuery { + LinkedList processBetweenHdfsFileMetadata(HdfsOffset startOffset, HdfsOffset endOffset); // TODO: PRESERVED IN REBASE, maybe rename it while at it. + void commit(HdfsOffset offset); // TODO: PRESERVED IN REBASE + JsonArray hdfsOffsetMapToJSON(); // TODO: PRESERVED IN REBASE! + HdfsOffset getBeginningOffsets(); // TODO: NEW! This method returns the starting offsets for all available (aka. filtered) topic partitions. + HdfsOffset getInitialEndOffsets(); // TODO: NEW! This method returns the end offsets for all available (aka. filtered) topic partitions. +} \ No newline at end of file diff --git a/src/main/java/com/teragrep/pth_06/planner/HdfsQueryProcessor.java b/src/main/java/com/teragrep/pth_06/planner/HdfsQueryProcessor.java new file mode 100644 index 00000000..10a5e8c0 --- /dev/null +++ b/src/main/java/com/teragrep/pth_06/planner/HdfsQueryProcessor.java @@ -0,0 +1,163 @@ +package com.teragrep.pth_06.planner; + +import com.google.gson.JsonArray; +import com.google.gson.JsonObject; +import com.teragrep.pth_06.config.Config; +import com.teragrep.pth_06.HdfsTopicPartitionOffsetMetadata; +import com.teragrep.pth_06.planner.offset.HdfsOffset; +import com.teragrep.pth_06.planner.walker.HdfsConditionWalker; +import org.apache.kafka.common.TopicPartition; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.Map; + +// Class for processing hdfs queries. +public class HdfsQueryProcessor implements HdfsQuery { + private final Logger LOGGER = LoggerFactory.getLogger(HdfsQueryProcessor.class); + private LinkedList topicPartitionList; + private final HdfsDBClient hdfsDBClient; + private String topicsRegexString; + private Map hdfsOffsetMap; + + public HdfsQueryProcessor(Config config) { + // Filter only topics using regex pattern + topicsRegexString = null; + if (config.query != null) { + try { + HdfsConditionWalker parser = new HdfsConditionWalker(); + topicsRegexString = parser.fromString(config.query); + } catch (Exception e) { + throw new RuntimeException("HdfsQueryProcessor problems when construction Query conditions query:" + config.query + " exception:" + e); + } + } + if (topicsRegexString == null) { + topicsRegexString = "^.*$"; // all topics if none given + } + // Implement hdfs db client that fetches the metadata for the files that are stored in hdfs based on topic name (aka. directory containing the files for a specific topic in HDFS). + // Remember to implement Kerberized HDFS access for prod. Tests are done using normal access on mini cluster. + try { + this.hdfsDBClient = new HdfsDBClient( + config, + topicsRegexString // topicsRegexString only searches for the given topic/topics (aka. directories). + ); + } catch (Exception e) { + throw new RuntimeException(e); + } + + // Generate the new starting partition offsets for kafka. + hdfsOffsetMap = new HashMap<>(); // This parameter is used for generating the new start offsets for the KafkaOffsetPlanner. hdfsOffsetMapToJSON() is used to transfer the parameter using printer. + seekToResults(); // This method loads all the available metadata to TopicPartitionList from HDFS. + // Create a map that only contains the metadata with the highest offset for every topic partition. + for (HdfsTopicPartitionOffsetMetadata r : topicPartitionList) { + long partitionStart = r.endOffset; + if (!hdfsOffsetMap.containsKey(r.topicPartition)) { + hdfsOffsetMap.put(r.topicPartition, partitionStart+1); + } else { + if (hdfsOffsetMap.get(r.topicPartition) < partitionStart) { + hdfsOffsetMap.replace(r.topicPartition, partitionStart+1); + } + } + } + LOGGER.debug("HdfsQueryProcessor.HdfsQueryProcessor>"); + } + + // pulls the metadata for available topic partition files in HDFS. + private void seekToResults() { + LOGGER.debug("HdfsQueryProcessor.seekToResults>"); + try { + topicPartitionList = hdfsDBClient.pullToPartitionList(); // queries the list of topic partitions based on the topic name condition filtering. + LOGGER.debug("HdfsQueryProcessor.seekToResults>"); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + // Returns all the available HDFS file metadata between the given topic partition offsets. + @Override + public LinkedList processBetweenHdfsFileMetadata(HdfsOffset startOffset, HdfsOffset endOffset) { + LinkedList rv = new LinkedList<>(); + Map endOffsetMap = endOffset.getOffsetMap(); + Map startOffsetMap = startOffset.getOffsetMap(); + for (HdfsTopicPartitionOffsetMetadata r : topicPartitionList) { + if ( (endOffsetMap.get(r.topicPartition) >= r.endOffset) & (startOffsetMap.get(r.topicPartition) <= r.endOffset) ) { + rv.add(new HdfsTopicPartitionOffsetMetadata(r.topicPartition, r.endOffset, r.hdfsFilePath, r.hdfsFileSize)); + } + } + return rv; + } + + // Removes the committed topic partition offsets from topicPartitionList. + @Override + public void commit(HdfsOffset offset) { + Map offsetMap = offset.getOffsetMap(); + LinkedList newTopicPartitionList = new LinkedList<>(); + // Generate new topicPartitionList where the metadata with offset values lower than the offset values given as input parameter are filtered out. + for (HdfsTopicPartitionOffsetMetadata r : topicPartitionList) { + if (offsetMap.get(r.topicPartition) < r.endOffset) { + newTopicPartitionList.add(new HdfsTopicPartitionOffsetMetadata(r.topicPartition, r.endOffset, r.hdfsFilePath, r.hdfsFileSize)); + } + } + topicPartitionList = newTopicPartitionList; + } + + // Prints json array containing the starting offsets for kafka to use. + @Override + public JsonArray hdfsOffsetMapToJSON() { + JsonArray ja = new JsonArray(); + for (Map.Entry entry : hdfsOffsetMap.entrySet()) { + String topic = entry.getKey().topic(); + String partition = String.valueOf(entry.getKey().partition()); + String offset = String.valueOf(entry.getValue()); + JsonObject jo = new JsonObject(); + jo.addProperty("topic", topic); + jo.addProperty("partition", partition); + jo.addProperty("offset", offset); + ja.add(jo); + } + return ja; + } + + // returns the starting offsets for all available (aka. filtered) topic partitions. + @Override + public HdfsOffset getBeginningOffsets() { + Map startOffset = new HashMap<>(); + // Go through the topicPartitionList to generate start offsets. + for (HdfsTopicPartitionOffsetMetadata r : topicPartitionList) { + // Generate startOffset + // When going through the result, store the topic partition with the lowest offset to the startOffset object. + long partitionOffset = r.endOffset; + if (!startOffset.containsKey(r.topicPartition)) { + startOffset.put(r.topicPartition, partitionOffset); + } else { + if (startOffset.get(r.topicPartition) > partitionOffset) { + startOffset.replace(r.topicPartition, partitionOffset); + } + } + } + return new HdfsOffset(startOffset); + } + + // returns the end offsets for all available (aka. filtered) topic partitions. + @Override + public HdfsOffset getInitialEndOffsets() { + Map endOffset = new HashMap<>(); + // Go through the topicPartitionList to generate end offsets. + for (HdfsTopicPartitionOffsetMetadata r : topicPartitionList) { + long partitionOffset = r.endOffset; + // Generate endOffset + // When going through the result, store the topic partition with the highest offset to the endOffset object. + if (!endOffset.containsKey(r.topicPartition)) { + endOffset.put(r.topicPartition, partitionOffset); + } else { + if (endOffset.get(r.topicPartition) < partitionOffset) { + endOffset.replace(r.topicPartition, partitionOffset); + } + } + } + return new HdfsOffset(endOffset); + } +} \ No newline at end of file diff --git a/src/main/java/com/teragrep/pth_06/planner/KafkaQuery.java b/src/main/java/com/teragrep/pth_06/planner/KafkaQuery.java index fafca70b..88988eb6 100644 --- a/src/main/java/com/teragrep/pth_06/planner/KafkaQuery.java +++ b/src/main/java/com/teragrep/pth_06/planner/KafkaQuery.java @@ -45,6 +45,7 @@ */ package com.teragrep.pth_06.planner; +import com.google.gson.JsonArray; import com.teragrep.pth_06.planner.offset.KafkaOffset; import org.apache.kafka.common.TopicPartition; @@ -65,4 +66,8 @@ public interface KafkaQuery { Map getBeginningOffsets(KafkaOffset endOffset); void commit(KafkaOffset offset); + + void seekToHdfsOffsets(JsonArray hdfsStartOffsets); + + Map getConsumerPositions(JsonArray startOffsets); } diff --git a/src/main/java/com/teragrep/pth_06/planner/KafkaQueryProcessor.java b/src/main/java/com/teragrep/pth_06/planner/KafkaQueryProcessor.java index 7c123c8c..a3a1f1ba 100644 --- a/src/main/java/com/teragrep/pth_06/planner/KafkaQueryProcessor.java +++ b/src/main/java/com/teragrep/pth_06/planner/KafkaQueryProcessor.java @@ -46,6 +46,9 @@ package com.teragrep.pth_06.planner; import com.google.common.annotations.VisibleForTesting; +import com.google.gson.JsonArray; +import com.google.gson.JsonElement; +import com.google.gson.JsonObject; import com.teragrep.pth_06.config.Config; import com.teragrep.pth_06.planner.offset.KafkaOffset; import com.teragrep.pth_06.planner.walker.KafkaWalker; @@ -261,4 +264,37 @@ public Map getBeginningOffsets(KafkaOffset endOffset) { public void commit(KafkaOffset offset) { // no-op } + + @Override + public void seekToHdfsOffsets(JsonArray startOffsets) { + // startOffsets holds the new starting offsets for Kafka in JsonArray format: + // [{"topic":"testConsumerTopic", "partition":"7", "offset":"13"}, {"topic":"testConsumerTopic", "partition":"8", "offset":"13"}, {"topic":"testConsumerTopic", "partition":"5", "offset":"13"}, {"topic":"testConsumerTopic", "partition":"6", "offset":"13"}, {"topic":"testConsumerTopic", "partition":"3", "offset":"13"}, {"topic":"testConsumerTopic", "partition":"4", "offset":"13"}, {"topic":"testConsumerTopic", "partition":"1", "offset":"13"}, {"topic":"testConsumerTopic", "partition":"2", "offset":"13"}, {"topic":"testConsumerTopic", "partition":"0", "offset":"13"}, {"topic":"testConsumerTopic", "partition":"9", "offset":"13"}] + // Deserialize JsonArray back to Map: + Map hdfsStartOffsets = new HashMap<>(); + for (JsonElement pa : startOffsets) { + JsonObject offsetObject = pa.getAsJsonObject(); + TopicPartition topicPartition = new TopicPartition(offsetObject.get("topic").getAsString(), offsetObject.get("partition").getAsInt()); + Long offset = offsetObject.get("offset").getAsLong(); + hdfsStartOffsets.put(topicPartition, offset); + } + + // Seek the consumer to the new starting offsets defined by startOffsets for all the topic partitions. + for (TopicPartition topicPartition : topicPartitionSet) { + if (hdfsStartOffsets.containsKey(topicPartition)) { + kafkaConsumer.seek(topicPartition, hdfsStartOffsets.get(topicPartition)); + } + } + + } + + @Override + public Map getConsumerPositions(JsonArray startOffsets) { + // Get the consumer position for all available topic partitions. + Map topicPartitionStartOffsetMap = new HashMap<>(); + for (TopicPartition topicPartition : topicPartitionSet) { + long position = kafkaConsumer.position(topicPartition, Duration.ofSeconds(60)); + topicPartitionStartOffsetMap.put(topicPartition, position); + } + return topicPartitionStartOffsetMap; + } } diff --git a/src/main/java/com/teragrep/pth_06/planner/offset/DatasourceOffset.java b/src/main/java/com/teragrep/pth_06/planner/offset/DatasourceOffset.java index 7e686bb9..c24dd9b7 100644 --- a/src/main/java/com/teragrep/pth_06/planner/offset/DatasourceOffset.java +++ b/src/main/java/com/teragrep/pth_06/planner/offset/DatasourceOffset.java @@ -63,10 +63,26 @@ public class DatasourceOffset extends Offset implements Serializable { SerializedDatasourceOffset serializedDatasourceOffset; + public DatasourceOffset(HdfsOffset hdfsOffset, LongOffset archiveOffset, KafkaOffset kafkaOffset) { + this.serializedDatasourceOffset = new SerializedDatasourceOffset(hdfsOffset, archiveOffset, kafkaOffset); + } + + public DatasourceOffset(HdfsOffset hdfsOffset, KafkaOffset kafkaOffset) { + this.serializedDatasourceOffset = new SerializedDatasourceOffset(hdfsOffset, kafkaOffset); + } + + public DatasourceOffset(HdfsOffset hdfsOffset, LongOffset archiveOffset) { + this.serializedDatasourceOffset = new SerializedDatasourceOffset(hdfsOffset, archiveOffset); + } + public DatasourceOffset(LongOffset archiveOffset, KafkaOffset kafkaOffset) { this.serializedDatasourceOffset = new SerializedDatasourceOffset(archiveOffset, kafkaOffset); } + public DatasourceOffset(HdfsOffset hdfsOffset) { + this.serializedDatasourceOffset = new SerializedDatasourceOffset(hdfsOffset); + } + public DatasourceOffset(LongOffset archiveOffset) { this.serializedDatasourceOffset = new SerializedDatasourceOffset(archiveOffset); } @@ -80,6 +96,10 @@ public DatasourceOffset(String s) { this.serializedDatasourceOffset = gson.fromJson(s, SerializedDatasourceOffset.class); } + public HdfsOffset getHdfsOffset() { + return serializedDatasourceOffset.hdfsOffset; + } + public LongOffset getArchiveOffset() { return serializedDatasourceOffset.archiveOffset; } diff --git a/src/main/java/com/teragrep/pth_06/planner/offset/HdfsOffset.java b/src/main/java/com/teragrep/pth_06/planner/offset/HdfsOffset.java new file mode 100644 index 00000000..5a8e73c6 --- /dev/null +++ b/src/main/java/com/teragrep/pth_06/planner/offset/HdfsOffset.java @@ -0,0 +1,63 @@ +package com.teragrep.pth_06.planner.offset; + +import com.google.common.reflect.TypeToken; +import com.google.gson.Gson; +import org.apache.kafka.common.TopicPartition; +import org.apache.spark.sql.connector.read.streaming.Offset; + +import java.io.Serializable; +import java.lang.reflect.Type; +import java.util.HashMap; +import java.util.Map; + +// Class for representing a serializable offset of HDFS data source. +// S3 has epoch hours as offsets, kafka has native TopicPartition offsets and HDFS should have file-metadata (use same format as in Kafka, topicpartition + record offset, which can be extracted from the metadata). + +public class HdfsOffset extends Offset implements Serializable { + + // TODO: Implement everything that is needed for tracking the offsets for HDFS datasource. + + private static final Type mapType = new TypeToken>() {}.getType(); + private final Map serializedHdfsOffset; + + public HdfsOffset(Map offset) { + serializedHdfsOffset = new HashMap<>(offset.size()); + for (Map.Entry entry : offset.entrySet()) { + + serializedHdfsOffset.put(entry.getKey().toString(), entry.getValue()); // offset + } + } + + // TODO: Check if these methods originating from KafkaOffset can be implemented as-is or not. + public HdfsOffset(String s) { + Gson gson = new Gson(); + serializedHdfsOffset = gson.fromJson(s, mapType); + } + public Map getOffsetMap() { + Map rv = new HashMap<>(serializedHdfsOffset.size()); + + for (Map.Entry entry : serializedHdfsOffset.entrySet()) { + String topicAndPartition = entry.getKey(); + long offset = entry.getValue(); + + int splitterLocation = topicAndPartition.lastIndexOf('-'); + int partition = Integer.parseInt(topicAndPartition.substring(splitterLocation + 1)); + String topic = topicAndPartition.substring(0, splitterLocation); + rv.put(new TopicPartition(topic, partition), offset); + } + + return rv; + } + + @Override + public String json() { + Gson gson = new Gson(); + return gson.toJson(serializedHdfsOffset); + } + @Override + public String toString() { + return "HdfsOffset{" + + "serializedHdfsOffset=" + serializedHdfsOffset + + '}'; + } +} \ No newline at end of file diff --git a/src/main/java/com/teragrep/pth_06/planner/offset/SerializedDatasourceOffset.java b/src/main/java/com/teragrep/pth_06/planner/offset/SerializedDatasourceOffset.java index 8dd1e37f..48be8f57 100644 --- a/src/main/java/com/teragrep/pth_06/planner/offset/SerializedDatasourceOffset.java +++ b/src/main/java/com/teragrep/pth_06/planner/offset/SerializedDatasourceOffset.java @@ -61,27 +61,64 @@ public class SerializedDatasourceOffset implements Serializable { private final Long version = 1L; + public final HdfsOffset hdfsOffset; public final LongOffset archiveOffset; public final KafkaOffset kafkaOffset; - public SerializedDatasourceOffset(LongOffset archiveOffset, KafkaOffset kafkaOffset) { + public SerializedDatasourceOffset(HdfsOffset hdfsOffset, LongOffset archiveOffset, KafkaOffset kafkaOffset) { + this.hdfsOffset = hdfsOffset; this.archiveOffset = archiveOffset; this.kafkaOffset = kafkaOffset; } + public SerializedDatasourceOffset( + LongOffset archiveOffset, + KafkaOffset kafkaOffset + ) { + this.hdfsOffset = null; + this.archiveOffset = archiveOffset; + this.kafkaOffset = kafkaOffset; + } + + public SerializedDatasourceOffset( + HdfsOffset hdfsOffset, + LongOffset archiveOffset + ) { + this.hdfsOffset = hdfsOffset; + this.archiveOffset = archiveOffset; + this.kafkaOffset = null; + } + + public SerializedDatasourceOffset( + HdfsOffset hdfsOffset, + KafkaOffset kafkaOffset + ) { + this.hdfsOffset = hdfsOffset; + this.archiveOffset = null; + this.kafkaOffset = kafkaOffset; + } + + public SerializedDatasourceOffset(HdfsOffset hdfsOffset) { + this.hdfsOffset = hdfsOffset; + this.kafkaOffset = null; + this.archiveOffset = null; + } + public SerializedDatasourceOffset(LongOffset archiveOffset) { + this.hdfsOffset = null; this.kafkaOffset = null; this.archiveOffset = archiveOffset; } public SerializedDatasourceOffset(KafkaOffset kafkaOffset) { + this.hdfsOffset = null; this.kafkaOffset = kafkaOffset; this.archiveOffset = null; } @Override public String toString() { - return "SerializedDatasourceOffset{" + "version=" + version + ", archiveOffset=" + archiveOffset + return "SerializedDatasourceOffset{" + "version=" + version + ", hdfsOffset" + hdfsOffset + ", archiveOffset=" + archiveOffset + ", kafkaOffset=" + kafkaOffset + '}'; } } diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/HdfsConditionWalker.java b/src/main/java/com/teragrep/pth_06/planner/walker/HdfsConditionWalker.java new file mode 100644 index 00000000..dc0ae39a --- /dev/null +++ b/src/main/java/com/teragrep/pth_06/planner/walker/HdfsConditionWalker.java @@ -0,0 +1,50 @@ +package com.teragrep.pth_06.planner.walker; + +import com.teragrep.jue_01.GlobToRegEx; +import org.w3c.dom.Element; + +// HDFS planner walker, only aims to filter out the topics as the planner only outputs the metadata for AVRO-files containing the records. The rest of the condition handling is done in the separate tasker walker. +public class HdfsConditionWalker extends XmlWalker{ + + @Override + String emitElem(Element current) { + String tag = current.getTagName(); + String value = current.getAttribute("value"); + String operation = current.getAttribute("operation"); + + String queryCondition = null; + // only index equals supported + if (tag.equalsIgnoreCase("index")) { + if (operation.equalsIgnoreCase("EQUALS")) { + queryCondition = GlobToRegEx.regexify(value); + } + } + return queryCondition; + } + + public String fromString(String inXml) throws Exception { + return super.fromString(inXml); + } + + @Override + String emitLogicalOperation(String op, Object l, Object r) throws Exception { + String left = (String) l; + String right = (String) r; + + String rv = null; + /* + index can not have two values at the same go therefore "AND".equals(op) + is not implemented + */ + if ("OR".equals(op)) { + rv = "(" + left + "|" + right + ")"; + } + return rv; + } + + @Override + String emitUnaryOperation(String op, Element current) throws Exception { + // NOT is a filter, not a topic matcher + return null; + } +} \ No newline at end of file diff --git a/src/main/java/com/teragrep/pth_06/scheduler/Batch.java b/src/main/java/com/teragrep/pth_06/scheduler/Batch.java index e3bf3eb4..2665b2f3 100644 --- a/src/main/java/com/teragrep/pth_06/scheduler/Batch.java +++ b/src/main/java/com/teragrep/pth_06/scheduler/Batch.java @@ -47,6 +47,7 @@ import com.teragrep.pth_06.config.Config; import com.teragrep.pth_06.planner.ArchiveQuery; +import com.teragrep.pth_06.planner.HdfsQuery; import com.teragrep.pth_06.planner.KafkaQuery; import org.apache.spark.sql.connector.read.streaming.Offset; import org.slf4j.Logger; @@ -67,17 +68,18 @@ public final class Batch extends LinkedList> { private long numberOfBatches = 0; private final LinkedList runQueueArray; private final Config config; + private final HdfsQuery hdfsQuery; private final ArchiveQuery archiveQuery; private final KafkaQuery kafkaQuery; - public Batch(Config config, ArchiveQuery aq, KafkaQuery kq) { + public Batch(Config config, HdfsQuery hq, ArchiveQuery aq, KafkaQuery kq) { this.config = config; this.runQueueArray = new LinkedList<>(); for (int i = 0; i < config.batchConfig.numPartitions; i++) { this.runQueueArray.add(new BatchTaskQueue()); } - + this.hdfsQuery = hq; this.archiveQuery = aq; this.kafkaQuery = kq; } @@ -86,8 +88,17 @@ public Batch processRange(Offset start, Offset end) { LOGGER.debug("processRange"); BatchSliceCollection slice = null; + + if (config.isHdfsEnabled) { + slice = new HdfsBatchSliceCollection(this.hdfsQuery).processRange(start, end); + } + if (config.isArchiveEnabled) { - slice = new ArchiveBatchSliceCollection(this.archiveQuery).processRange(start, end); + if (slice == null) { + slice = new ArchiveBatchSliceCollection(this.archiveQuery).processRange(start, end); + } else { + slice.addAll(new ArchiveBatchSliceCollection(this.archiveQuery).processRange(start, end)); + } } if (config.isKafkaEnabled) { diff --git a/src/main/java/com/teragrep/pth_06/scheduler/BatchSlice.java b/src/main/java/com/teragrep/pth_06/scheduler/BatchSlice.java index 330eabb7..7271722b 100644 --- a/src/main/java/com/teragrep/pth_06/scheduler/BatchSlice.java +++ b/src/main/java/com/teragrep/pth_06/scheduler/BatchSlice.java @@ -46,6 +46,7 @@ package com.teragrep.pth_06.scheduler; import com.teragrep.pth_06.ArchiveS3ObjectMetadata; +import com.teragrep.pth_06.HdfsTopicPartitionOffsetMetadata; import com.teragrep.pth_06.KafkaTopicPartitionOffsetMetadata; import java.io.Serializable; @@ -62,23 +63,34 @@ public final class BatchSlice implements Serializable { public enum Type { - ARCHIVE, KAFKA + ARCHIVE, KAFKA, HDFS } public final Type type; public final ArchiveS3ObjectMetadata archiveS3ObjectMetadata; public final KafkaTopicPartitionOffsetMetadata kafkaTopicPartitionOffsetMetadata; + public final HdfsTopicPartitionOffsetMetadata hdfsTopicPartitionOffsetMetadata; public BatchSlice(ArchiveS3ObjectMetadata archiveS3ObjectMetadata) { this.type = Type.ARCHIVE; this.archiveS3ObjectMetadata = archiveS3ObjectMetadata; this.kafkaTopicPartitionOffsetMetadata = null; + this.hdfsTopicPartitionOffsetMetadata = null; } public BatchSlice(KafkaTopicPartitionOffsetMetadata kafkaTopicPartitionOffsetMetadata) { this.type = Type.KAFKA; this.archiveS3ObjectMetadata = null; this.kafkaTopicPartitionOffsetMetadata = kafkaTopicPartitionOffsetMetadata; + this.hdfsTopicPartitionOffsetMetadata = null; + } + + public BatchSlice(HdfsTopicPartitionOffsetMetadata hdfsTopicPartitionOffsetMetadata) { + this.type = Type.HDFS; + this.archiveS3ObjectMetadata = null; + this.kafkaTopicPartitionOffsetMetadata = null; + this.hdfsTopicPartitionOffsetMetadata = hdfsTopicPartitionOffsetMetadata; + } public long getSize() { @@ -95,6 +107,13 @@ public long getSize() { case KAFKA: // TODO estimate based on offset delta return 1024 * 1024 * 16; + case HDFS: + // The files in HDFS have a default size limiter set to them during creation. Use that limit (64000000) or hdfsTopicPartitionOffsetMetadata.getFileSize(). + if (hdfsTopicPartitionOffsetMetadata != null) { + return hdfsTopicPartitionOffsetMetadata.hdfsFileSize; + } else { + throw new RuntimeException("Expected HDFS file metadata, instead was null"); + } default: throw new IllegalStateException("unknown BatchSliceType " + type); } @@ -103,6 +122,6 @@ public long getSize() { @Override public String toString() { return "BatchSlice{" + "batchSliceType=" + type + ", archiveS3ObjectMetadata=" + archiveS3ObjectMetadata - + ", kafkaTopicPartitionOffsetMetadata=" + kafkaTopicPartitionOffsetMetadata + '}'; + + ", kafkaTopicPartitionOffsetMetadata=" + kafkaTopicPartitionOffsetMetadata + ", hdfsTopicPartitionOffsetMetadata=" + hdfsTopicPartitionOffsetMetadata + '}'; } } diff --git a/src/main/java/com/teragrep/pth_06/scheduler/HdfsBatchSliceCollection.java b/src/main/java/com/teragrep/pth_06/scheduler/HdfsBatchSliceCollection.java new file mode 100644 index 00000000..5ae1ebc0 --- /dev/null +++ b/src/main/java/com/teragrep/pth_06/scheduler/HdfsBatchSliceCollection.java @@ -0,0 +1,71 @@ +package com.teragrep.pth_06.scheduler; + +import com.teragrep.pth_06.HdfsTopicPartitionOffsetMetadata; +import com.teragrep.pth_06.planner.HdfsQuery; +import com.teragrep.pth_06.planner.KafkaQuery; +import com.teragrep.pth_06.planner.offset.DatasourceOffset; +import com.teragrep.pth_06.planner.offset.HdfsOffset; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.spark.sql.connector.read.streaming.Offset; + +import java.util.LinkedList; + +public final class HdfsBatchSliceCollection extends BatchSliceCollection { + private final Logger LOGGER = LoggerFactory.getLogger(HdfsBatchSliceCollection.class); + private final HdfsQuery hq; + + public HdfsBatchSliceCollection(HdfsQuery hq) { + super(); + this.hq = hq; + } + + public HdfsBatchSliceCollection processRange(Offset start, Offset end) { + /* + KAFKA: + KafkaOffset kafkaStartOffset = ((DatasourceOffset)start).getKafkaOffset(); + KafkaOffset kafkaEndOffset = ((DatasourceOffset)end).getKafkaOffset(); + KafkaBatchSliceCollection rv = generate(kafkaStartOffset, kafkaEndOffset); + LOGGER.debug("processRange(): arg start " + start + " arg end: " + end + " rv: " + rv ); + return rv; + + ARCHIVE: + LOGGER.debug("processRange(): args: start: " + start + " end: " + end); + this.clear(); // clear internal list + Result> + result = aq.processBetweenUnixEpochHours(((DatasourceOffset)start).getArchiveOffset().offset(), + ((DatasourceOffset)end).getArchiveOffset().offset()); + for (Record r : result) { + this.add( + new BatchSlice( + new ArchiveS3ObjectMetadata( + r.get(0, String.class), // id + r.get(6, String.class), // bucket + r.get(7, String.class), // path + r.get(1, String.class), // directory + r.get(2, String.class), // stream + r.get(3, String.class), // host + r.get(8, Long.class), // logtime + r.get(9, Long.class) // compressedSize + ) + ) + ); + } + return this; + * */ + + // HDFS: + // If the slices are not distributed correctly, refactor the code to use the archive approach instead of kafka approach. + LOGGER.debug("processRange(): args: start: " + start + " end: " + end); + HdfsOffset hdfsStartOffset = ((DatasourceOffset)start).getHdfsOffset(); + HdfsOffset hdfsEndOffset = ((DatasourceOffset)end).getHdfsOffset(); + LinkedList result = hq.processBetweenHdfsFileMetadata(hdfsStartOffset, hdfsEndOffset); + for (HdfsTopicPartitionOffsetMetadata r : result) { + this.add( + new BatchSlice(r) + ); + } + return this; + } + +} \ No newline at end of file diff --git a/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartition.java b/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartition.java new file mode 100644 index 00000000..16980db3 --- /dev/null +++ b/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartition.java @@ -0,0 +1,4 @@ +package com.teragrep.pth_06.task; + +public class HdfsMicroBatchInputPartition { +} diff --git a/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReader.java b/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReader.java new file mode 100644 index 00000000..f1eef110 --- /dev/null +++ b/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReader.java @@ -0,0 +1,4 @@ +package com.teragrep.pth_06.task; + +public class HdfsMicroBatchInputPartitionReader { +} diff --git a/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReader.java b/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReader.java new file mode 100644 index 00000000..6808aa6a --- /dev/null +++ b/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReader.java @@ -0,0 +1,4 @@ +package com.teragrep.pth_06.task.hdfs; + +public class AvroReader { +} diff --git a/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConverter.java b/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConverter.java new file mode 100644 index 00000000..afd23590 --- /dev/null +++ b/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConverter.java @@ -0,0 +1,4 @@ +package com.teragrep.pth_06.task.hdfs; + +public class HdfsRecordConverter { +} diff --git a/src/test/java/com/teragrep/pth_06/InstantiationTest.java b/src/test/java/com/teragrep/pth_06/InstantiationTest.java index 971a910b..2c13700b 100644 --- a/src/test/java/com/teragrep/pth_06/InstantiationTest.java +++ b/src/test/java/com/teragrep/pth_06/InstantiationTest.java @@ -54,6 +54,7 @@ import com.google.gson.Gson; import com.google.gson.JsonObject; import com.teragrep.pth_06.planner.MockDBData; +import com.teragrep.pth_06.planner.MockHDFS; import com.teragrep.pth_06.planner.MockKafkaConsumerFactory; import com.teragrep.pth_06.task.s3.MockS3; import com.teragrep.pth_06.task.s3.Pth06S3Client; @@ -96,6 +97,10 @@ public class InstantiationTest { private long expectedRows = 0L; + private final String hdfsPath = "hdfs:///opt/teragrep/cfe_39/srv/"; + private String hdfsUri; // Can only be defined after starting the mock hdfs. + private final MockHDFS mockHDFS = new MockHDFS(hdfsPath); + @BeforeAll public void prepareEnv() throws Exception { //Logger.getRootLogger().setLevel(Level.ERROR); @@ -104,6 +109,9 @@ public void prepareEnv() throws Exception { mockS3.start(); + // Start mock hdfs here in a similar way that the mockS3 is implemented. + hdfsUri = mockHDFS.startMiniCluster(); + spark = SparkSession .builder() .appName("Java Spark SQL basic example") @@ -115,7 +123,10 @@ public void prepareEnv() throws Exception { //spark.sparkContext().setLogLevel("ERROR"); - expectedRows = preloadS3Data() + MockKafkaConsumerFactory.getNumRecords(); + expectedRows = preloadS3Data() + MockKafkaConsumerFactory.getNumRecords() - 9; + /* Remove the - modifier once the HDFS tasker is up and running. + If file 0.8 is stored to HDFS, 9 records out of 14 will be read from HDFS instead of Kafka. + If files 0.8 and 0.13 are stored to HDFS, all 14 records will be read from HDFS instead of Kafka.*/ } @Test @@ -134,7 +145,7 @@ public void fullScanTest() throws StreamingQueryException, TimeoutException { .option("DBstreamdbname", "mock") .option("DBjournaldbname", "mock") .option("num_partitions", "1") - .option("queryXML", "") + .option("queryXML", "") // Only affects HDFS execution in the current test configuration. // audit information .option("TeragrepAuditQuery", "index=f17") .option("TeragrepAuditReason", "test run at fullScanTest()") @@ -147,6 +158,12 @@ public void fullScanTest() throws StreamingQueryException, TimeoutException { .option("kafka.sasl.jaas.config", "") .option("kafka.useMockKafkaConsumer", "true") .option("spark.cleaner.referenceTracking.cleanCheckpoints", "true") + // HDFS options + .option("hdfs.enabled", "true") + .option("hdfs.hdfsPath", hdfsPath) + .option("hdfs.hdfsUri", hdfsUri) + .option("hdfs.useMockHdfsDatabase", "true") + //.option("hdfs.hdfsCutoffOffset", Instant.now().toEpochMilli()) // Current time - pruning offset from cfe_39 + configurable cutoff offset. Granularity is in milliseconds. .load(); Dataset df2 = df.agg(functions.count("*")); diff --git a/src/test/java/com/teragrep/pth_06/MockTeragrepDatasource.java b/src/test/java/com/teragrep/pth_06/MockTeragrepDatasource.java index ac38bc0a..be310025 100644 --- a/src/test/java/com/teragrep/pth_06/MockTeragrepDatasource.java +++ b/src/test/java/com/teragrep/pth_06/MockTeragrepDatasource.java @@ -126,6 +126,13 @@ public StructType readSchema() { public MicroBatchStream toMicroBatchStream(String checkpointLocation) { Config config = new Config(options); + HdfsQuery hdfsQueryProcessor; + if (config.isHdfsEnabled) { + hdfsQueryProcessor = new HdfsQueryProcessor(config); + } else { + hdfsQueryProcessor = null; + } + ArchiveQuery archiveQueryProcessor = new MockArchiveQueryProcessor( "" ); @@ -140,7 +147,7 @@ public MicroBatchStream toMicroBatchStream(String checkpointLocation) { kafkaQueryProcessor = null; } - return new ArchiveMicroStreamReader(archiveQueryProcessor, kafkaQueryProcessor, config); + return new ArchiveMicroStreamReader(hdfsQueryProcessor, archiveQueryProcessor, kafkaQueryProcessor, config); } }; } diff --git a/src/test/java/com/teragrep/pth_06/mockHdfsFiles/0.13 b/src/test/java/com/teragrep/pth_06/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 diff --git a/src/test/java/com/teragrep/pth_06/mockHdfsFiles/0.8 b/src/test/java/com/teragrep/pth_06/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 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()); + /* This condition can be used for managing the test offsets between HDFS and Kafka. If 0.13 file is included in the HDFS, then Kafka won't have any new records available for reading. + If 0.13 is excluded, then records from 0 to 8 will be read from HDFS and records from 9 to 13 will be read from Kafka. + Implement a method to decide if all the files should be added to HDFS or not*/ + if (!avroFile.getName().equals("0.13")) { + 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(); + } + + public void simpleTest() { + try { + hdfsReadTest(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + public void hdfsReadTest() throws IOException { + Map hdfsStartOffsets = new HashMap<>(); + // Check that the files were properly written to HDFS with a read test. + String path = "hdfs:///opt/teragrep/cfe_39/srv/"; // This path parameter should only lead to the directory that holds the directories representing different Kafka topics. + // ====== 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); + } + FileStatus[] directoryStatuses = fs.listStatus(newDirectoryPath, pathFilter); + if (directoryStatuses.length > 0) { + for (FileStatus r : directoryStatuses) { + LOGGER.info("The path to the directory is: {}", r.getPath()); + LOGGER.info("The directory name is: {}", r.getPath().getName()); + 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]; + LOGGER.info("File belongs to topic: {}", topic); + LOGGER.info("File belongs to partition: {}", partition); + LOGGER.info("File has an offset of: {}", offset); + if (!hdfsStartOffsets.containsKey(new TopicPartition(topic, Integer.parseInt(partition)))) { + hdfsStartOffsets.put(new TopicPartition(topic, Integer.parseInt(partition)), Long.parseLong(offset)); + } else { + if (hdfsStartOffsets.get(new TopicPartition(topic, Integer.parseInt(partition))) < Long.parseLong(offset)) { + hdfsStartOffsets.replace(new TopicPartition(topic, Integer.parseInt(partition)), Long.parseLong(offset)); + } + } + } + } + }else { + LOGGER.info("No matching directories found"); + } + LOGGER.info("hdfsStartOffsets.toString(): "); + LOGGER.info(hdfsStartOffsets.toString()); + + LinkedList rv = new LinkedList<>(); + JsonArray ja = new JsonArray(); + // Serializing the hdfsStartOffsets to a JsonArray ja. + for (Map.Entry entry : hdfsStartOffsets.entrySet()) { + String topic = entry.getKey().topic(); + String partition = String.valueOf(entry.getKey().partition()); + String offset = String.valueOf(entry.getValue()); + rv.add(String.format( + "{\"topic\":\"%s\", \"partition\":\"%s\", \"offset\":\"%s\"}", + topic, partition, offset + )); + JsonObject jo = new JsonObject(); // TODO: Use this instead of string + jo.addProperty("topic", topic); + jo.addProperty("partition", partition); + jo.addProperty("offset", offset); + ja.add(jo); + } + // LOGGER.info(rv.toString()); + LOGGER.info("ja.toString(): "); + LOGGER.info(ja.toString()); + + // Deserialize ja back to Map + Map offsetMap = new HashMap<>(); + for (JsonElement pa : ja) { + JsonObject offsetObject = pa.getAsJsonObject(); + TopicPartition topicPartition = new TopicPartition(offsetObject.get("topic").getAsString(), offsetObject.get("partition").getAsInt()); + Long offset = offsetObject.get("offset").getAsLong(); + offsetMap.put(topicPartition, offset); + } + + LOGGER.info("offsetMap.toString(): "); + LOGGER.info(offsetMap.toString()); + // Assert that hdfsStartOffsets and offsetMap are identical after the json serialization and deserialization cycle of hdfsStartOffsets. + Assertions.assertEquals(hdfsStartOffsets, offsetMap); + + // ja.toString() now outputs: + // [{"topic":"testConsumerTopic","partition":"7","offset":"13"},{"topic":"testConsumerTopic","partition":"8","offset":"13"},{"topic":"testConsumerTopic","partition":"5","offset":"13"},{"topic":"testConsumerTopic","partition":"6","offset":"13"},{"topic":"testConsumerTopic","partition":"3","offset":"13"},{"topic":"testConsumerTopic","partition":"4","offset":"13"},{"topic":"testConsumerTopic","partition":"1","offset":"13"},{"topic":"testConsumerTopic","partition":"2","offset":"13"},{"topic":"testConsumerTopic","partition":"0","offset":"13"},{"topic":"testConsumerTopic","partition":"9","offset":"13"}] + fs.close(); + } + + private static final PathFilter pathFilter = new PathFilter() { + @Override + public boolean accept(Path path) { + return path.getName().matches("^testConsumer.*$"); // Catches the directory names. + } + }; + +} From f82ed6cdde6025923f7477cb15466ed7d60c2d10 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Fri, 13 Sep 2024 09:40:58 +0300 Subject: [PATCH 02/62] Spotless --- pom.xml | 2 +- .../pth_06/ArchiveMicroStreamReader.java | 58 +++++----- .../HdfsTopicPartitionOffsetMetadata.java | 63 +++++++++-- .../com/teragrep/pth_06/config/Config.java | 3 +- .../teragrep/pth_06/config/HdfsConfig.java | 57 +++++++++- .../teragrep/pth_06/planner/HdfsDBClient.java | 66 ++++++++++- .../teragrep/pth_06/planner/HdfsQuery.java | 59 +++++++++- .../pth_06/planner/HdfsQueryProcessor.java | 105 +++++++++++++++--- .../pth_06/planner/KafkaQueryProcessor.java | 7 +- .../pth_06/planner/offset/HdfsOffset.java | 56 +++++++++- .../offset/SerializedDatasourceOffset.java | 19 +--- .../planner/walker/HdfsConditionWalker.java | 49 +++++++- .../com/teragrep/pth_06/scheduler/Batch.java | 3 +- .../teragrep/pth_06/scheduler/BatchSlice.java | 6 +- .../scheduler/HdfsBatchSliceCollection.java | 64 +++++++++-- .../task/HdfsMicroBatchInputPartition.java | 45 ++++++++ .../HdfsMicroBatchInputPartitionReader.java | 45 ++++++++ .../teragrep/pth_06/task/hdfs/AvroReader.java | 45 ++++++++ .../pth_06/task/hdfs/HdfsRecordConverter.java | 45 ++++++++ .../pth_06/MockTeragrepDatasource.java | 10 +- .../com/teragrep/pth_06/planner/MockHDFS.java | 101 +++++++++++++---- 21 files changed, 789 insertions(+), 119 deletions(-) diff --git a/pom.xml b/pom.xml index d9862b0e..04ae152e 100644 --- a/pom.xml +++ b/pom.xml @@ -29,9 +29,9 @@ -SNAPSHOT + 3.3.6 1.8 2.3.0 - 3.3.6 1.8 1.8 UTF-8 diff --git a/src/main/java/com/teragrep/pth_06/ArchiveMicroStreamReader.java b/src/main/java/com/teragrep/pth_06/ArchiveMicroStreamReader.java index c3d2bcbb..4afd75b8 100644 --- a/src/main/java/com/teragrep/pth_06/ArchiveMicroStreamReader.java +++ b/src/main/java/com/teragrep/pth_06/ArchiveMicroStreamReader.java @@ -105,7 +105,8 @@ public final class ArchiveMicroStreamReader implements MicroBatchStream { if (config.isHdfsEnabled) { this.hq = new HdfsQueryProcessor(config); hdfsOffsets = hq.hdfsOffsetMapToJSON(); - } else { + } + else { this.hq = null; hdfsOffsets = null; } @@ -121,7 +122,8 @@ public final class ArchiveMicroStreamReader implements MicroBatchStream { if (config.isHdfsEnabled) { this.kq = new KafkaQueryProcessor(config); this.kq.seekToHdfsOffsets(hdfsOffsets); - }else { + } + else { this.kq = new KafkaQueryProcessor(config); } } @@ -144,12 +146,11 @@ public final class ArchiveMicroStreamReader implements MicroBatchStream { if (this.hq != null && this.kq != null) { hdfsOffsets = this.hq.hdfsOffsetMapToJSON(); this.kq.seekToHdfsOffsets(hdfsOffsets); - } else { + } + else { hdfsOffsets = null; } - - LOGGER.debug("@VisibleForTesting MicroBatchReader> initialized"); } @@ -179,14 +180,14 @@ else if (!this.config.isArchiveEnabled && !this.config.isKafkaEnabled && this.co } else if (this.config.isArchiveEnabled && this.config.isKafkaEnabled && !this.config.isHdfsEnabled) { // kafka and archive - rv = new DatasourceOffset( + rv = new DatasourceOffset( new LongOffset(this.aq.getInitialOffset() - 3600L), new KafkaOffset(this.kq.getBeginningOffsets(null)) ); } else if (this.config.isArchiveEnabled && !this.config.isKafkaEnabled && this.config.isHdfsEnabled) { // archive and HDFS - rv = new DatasourceOffset( + rv = new DatasourceOffset( new HdfsOffset(this.hq.getBeginningOffsets().getOffsetMap()), new LongOffset(this.aq.getInitialOffset() - 3600L) ); @@ -198,7 +199,8 @@ else if (!this.config.isArchiveEnabled && this.config.isKafkaEnabled && this.con new HdfsOffset(this.hq.getBeginningOffsets().getOffsetMap()), new KafkaOffset(this.kq.getConsumerPositions(hdfsOffsets)) ); - } else { + } + else { rv = new DatasourceOffset( new HdfsOffset(this.hq.getBeginningOffsets().getOffsetMap()), new KafkaOffset(this.kq.getBeginningOffsets(null)) @@ -213,7 +215,8 @@ else if (this.config.isArchiveEnabled && this.config.isKafkaEnabled && this.conf new LongOffset(this.aq.getInitialOffset() - 3600L), new KafkaOffset(this.kq.getConsumerPositions(hdfsOffsets)) ); - }else { + } + else { rv = new DatasourceOffset( new HdfsOffset(this.hq.getBeginningOffsets().getOffsetMap()), new LongOffset(this.aq.getInitialOffset() - 3600L), @@ -242,7 +245,7 @@ public void commit(Offset offset) { this.aq.commit(((DatasourceOffset) offset).getArchiveOffset().offset()); } if (this.config.isHdfsEnabled) { - this.hq.commit(((DatasourceOffset)offset).getHdfsOffset()); + this.hq.commit(((DatasourceOffset) offset).getHdfsOffset()); } } @@ -269,35 +272,41 @@ public Offset latestOffset() { else if (!this.config.isArchiveEnabled && this.config.isKafkaEnabled && !this.config.isHdfsEnabled) { // only kafka rv = new DatasourceOffset(new KafkaOffset(this.kq.getInitialEndOffsets())); - } else if (!this.config.isArchiveEnabled && !this.config.isKafkaEnabled && this.config.isHdfsEnabled) { + } + else if (!this.config.isArchiveEnabled && !this.config.isKafkaEnabled && this.config.isHdfsEnabled) { // only hdfs rv = new DatasourceOffset(new HdfsOffset(this.hq.getInitialEndOffsets().getOffsetMap())); - } else if (this.config.isArchiveEnabled && this.config.isKafkaEnabled && !this.config.isHdfsEnabled) { + } + else if (this.config.isArchiveEnabled && this.config.isKafkaEnabled && !this.config.isHdfsEnabled) { // kafka and archive - rv = new DatasourceOffset( + rv = new DatasourceOffset( new LongOffset(this.aq.incrementAndGetLatestOffset()), new KafkaOffset(this.kq.getInitialEndOffsets()) ); - } else if (this.config.isArchiveEnabled && !this.config.isKafkaEnabled && this.config.isHdfsEnabled) { + } + else if (this.config.isArchiveEnabled && !this.config.isKafkaEnabled && this.config.isHdfsEnabled) { // archive and hdfs - rv = new DatasourceOffset( + rv = new DatasourceOffset( new HdfsOffset(this.hq.getInitialEndOffsets().getOffsetMap()), new LongOffset(this.aq.incrementAndGetLatestOffset()) ); - } else if (!this.config.isArchiveEnabled && this.config.isKafkaEnabled && this.config.isHdfsEnabled) { + } + else if (!this.config.isArchiveEnabled && this.config.isKafkaEnabled && this.config.isHdfsEnabled) { // Kafka and HDFS - rv = new DatasourceOffset( + rv = new DatasourceOffset( new HdfsOffset(this.hq.getInitialEndOffsets().getOffsetMap()), new KafkaOffset(this.kq.getInitialEndOffsets()) ); - } else if (this.config.isArchiveEnabled && this.config.isKafkaEnabled && this.config.isHdfsEnabled) { + } + else if (this.config.isArchiveEnabled && this.config.isKafkaEnabled && this.config.isHdfsEnabled) { // all three - rv = new DatasourceOffset( + rv = new DatasourceOffset( new HdfsOffset(this.hq.getInitialEndOffsets().getOffsetMap()), new LongOffset(this.aq.incrementAndGetLatestOffset()), new KafkaOffset(this.kq.getInitialEndOffsets()) ); - } else { + } + else { // none throw new IllegalStateException("no datasources enabled, can't get latest offset"); } @@ -361,13 +370,12 @@ public InputPartition[] planInputPartitions(Offset start, Offset end) { // Implement HdfsMicroBatchInputPartition usage here. If needed change the slice creation to be incremental like it is with archive, for that create HDFS variant of the incrementAndGetLatestOffset()-method from ArchiveQueryProcessor. // At the moment it fetches all the metadata available at once and puts them into hdfsTaskList. - /*inputPartitions.add(new HdfsMicroBatchInputPartition( - config.hdfsConfig, - hdfsTaskList - ));*/ + /*inputPartitions.add(new HdfsMicroBatchInputPartition( + config.hdfsConfig, + hdfsTaskList + ));*/ } - // kafka tasks for (BatchSlice batchSlice : taskObjectList) { if (batchSlice.type.equals(BatchSlice.Type.KAFKA)) { diff --git a/src/main/java/com/teragrep/pth_06/HdfsTopicPartitionOffsetMetadata.java b/src/main/java/com/teragrep/pth_06/HdfsTopicPartitionOffsetMetadata.java index 242ed316..31fea2e4 100644 --- a/src/main/java/com/teragrep/pth_06/HdfsTopicPartitionOffsetMetadata.java +++ b/src/main/java/com/teragrep/pth_06/HdfsTopicPartitionOffsetMetadata.java @@ -1,3 +1,48 @@ +/* + * Teragrep Archive Datasource (pth_06) + * 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.pth_06; import org.apache.kafka.common.TopicPartition; @@ -7,12 +52,18 @@ // Class for holding serializable metadata of HDFS files containing kafka records. // Maybe change the class name to something more appropriate. ie. HdfsFileMetadata public class HdfsTopicPartitionOffsetMetadata implements Serializable { + public final TopicPartition topicPartition; // Represents the Kafka topic partition which records the file contains. public final long endOffset; // Represents the offset of the record that was last added to the file. public final String hdfsFilePath; // Represents the file path where the file resides in HDFS. public final long hdfsFileSize; // Represents the size of the file in HDFS. Used for scheduling the batch slice. - public HdfsTopicPartitionOffsetMetadata(TopicPartition topicPartition, long offset, String filePath, long fileSize) { + public HdfsTopicPartitionOffsetMetadata( + TopicPartition topicPartition, + long offset, + String filePath, + long fileSize + ) { this.topicPartition = topicPartition; this.endOffset = offset; this.hdfsFilePath = filePath; @@ -21,11 +72,7 @@ public HdfsTopicPartitionOffsetMetadata(TopicPartition topicPartition, long offs @Override public String toString() { - return "HdfsTopicPartitionOffsetMetadata{" + - "topicPartition=" + topicPartition + - ", endOffset=" + endOffset + - ", hdfsFilePath=" + hdfsFilePath + - ", hdfsFileSize=" + hdfsFileSize + - '}'; + return "HdfsTopicPartitionOffsetMetadata{" + "topicPartition=" + topicPartition + ", endOffset=" + endOffset + + ", hdfsFilePath=" + hdfsFilePath + ", hdfsFileSize=" + hdfsFileSize + '}'; } -} \ No newline at end of file +} diff --git a/src/main/java/com/teragrep/pth_06/config/Config.java b/src/main/java/com/teragrep/pth_06/config/Config.java index d2659891..d2ef1851 100644 --- a/src/main/java/com/teragrep/pth_06/config/Config.java +++ b/src/main/java/com/teragrep/pth_06/config/Config.java @@ -98,7 +98,8 @@ public Config(Map opts) { isHdfsEnabled = opts.getOrDefault("hdfs.enabled", "false").equalsIgnoreCase("true"); if (isHdfsEnabled) { hdfsConfig = new HdfsConfig(opts); - } else { + } + else { hdfsConfig = new HdfsConfig(); } diff --git a/src/main/java/com/teragrep/pth_06/config/HdfsConfig.java b/src/main/java/com/teragrep/pth_06/config/HdfsConfig.java index 6b305f65..a1d1b4ec 100644 --- a/src/main/java/com/teragrep/pth_06/config/HdfsConfig.java +++ b/src/main/java/com/teragrep/pth_06/config/HdfsConfig.java @@ -1,3 +1,48 @@ +/* + * Teragrep Archive Datasource (pth_06) + * 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.pth_06.config; import java.time.Instant; @@ -12,8 +57,16 @@ public final class HdfsConfig { public final boolean useMockHdfsDatabase; // Represents the configuration parameter if mock database should be used or not. Used only for testing. public final boolean isStub; + public HdfsConfig(Map opts) { - cutoffOffset = Long.parseLong(opts.getOrDefault("hdfs.hdfsCutoffOffset", String.valueOf(Instant.now().toEpochMilli() - 72*3600000))); // Default is -72h from now + cutoffOffset = Long + .parseLong( + opts + .getOrDefault( + "hdfs.hdfsCutoffOffset", + String.valueOf(Instant.now().toEpochMilli() - 72 * 3600000) + ) + ); // Default is -72h from now hdfsPath = getOrThrow(opts, "hdfs.hdfsPath"); hdfsUri = getOrThrow(opts, "hdfs.hdfsUri"); String useMockHdfsDatabaseString = opts.getOrDefault("useMockHdfsDatabase", "false"); @@ -37,4 +90,4 @@ private String getOrThrow(Map opts, String key) { return value; } -} \ No newline at end of file +} diff --git a/src/main/java/com/teragrep/pth_06/planner/HdfsDBClient.java b/src/main/java/com/teragrep/pth_06/planner/HdfsDBClient.java index 9fa97d33..a45e091a 100644 --- a/src/main/java/com/teragrep/pth_06/planner/HdfsDBClient.java +++ b/src/main/java/com/teragrep/pth_06/planner/HdfsDBClient.java @@ -1,3 +1,48 @@ +/* + * Teragrep Archive Datasource (pth_06) + * 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.pth_06.planner; import com.teragrep.pth_06.config.Config; @@ -51,10 +96,12 @@ public HdfsDBClient(Config config, String topicsRegexString) 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.hdfsConfig.hdfsUri; // Get from config. @@ -63,7 +110,6 @@ public HdfsDBClient(Config config, String topicsRegexString) throws IOException // The values are fetched from config and input parameters (topic+partition+offset). path = config.hdfsConfig.hdfsPath; - // 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? @@ -120,23 +166,31 @@ public LinkedList pullToPartitionList() throws 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]; - HdfsTopicPartitionOffsetMetadata temp = new HdfsTopicPartitionOffsetMetadata(new TopicPartition(topic, Integer.parseInt(partition)), Integer.parseInt(offset), r2.getPath().toString(), r2.getLen()); + HdfsTopicPartitionOffsetMetadata temp = new HdfsTopicPartitionOffsetMetadata( + new TopicPartition(topic, Integer.parseInt(partition)), + Integer.parseInt(offset), + r2.getPath().toString(), + r2.getLen() + ); // Add the HdfsTopicPartitionOffsetMetadata object to the rv only if the file's modification timestamp is above ignoreBeforeEpoch. Timestamps are in milliseconds. if (r2.getModificationTime() >= ignoreBeforeEpoch) { rv.add(temp); } } } - }else { + } + else { LOGGER.info("No matching directories found"); } return rv; } + private final PathFilter topicFilter = new PathFilter() { + @Override public boolean accept(Path path) { return path.getName().matches(topicsRegexString); // Catches the directory names. } }; -} \ No newline at end of file +} diff --git a/src/main/java/com/teragrep/pth_06/planner/HdfsQuery.java b/src/main/java/com/teragrep/pth_06/planner/HdfsQuery.java index 3ba15afa..93fb3648 100644 --- a/src/main/java/com/teragrep/pth_06/planner/HdfsQuery.java +++ b/src/main/java/com/teragrep/pth_06/planner/HdfsQuery.java @@ -1,20 +1,71 @@ +/* + * Teragrep Archive Datasource (pth_06) + * 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.pth_06.planner; import com.google.gson.JsonArray; import com.teragrep.pth_06.HdfsTopicPartitionOffsetMetadata; import com.teragrep.pth_06.planner.offset.HdfsOffset; -import org.apache.kafka.common.TopicPartition; import java.util.LinkedList; -import java.util.Map; // TODO: Make an interface for for HDFS query. methods are placeholders at the moment. // TODO: Fetches semi-recent data from HDFS. public interface HdfsQuery { - LinkedList processBetweenHdfsFileMetadata(HdfsOffset startOffset, HdfsOffset endOffset); // TODO: PRESERVED IN REBASE, maybe rename it while at it. + + LinkedList processBetweenHdfsFileMetadata( + HdfsOffset startOffset, + HdfsOffset endOffset + ); // TODO: PRESERVED IN REBASE, maybe rename it while at it. + void commit(HdfsOffset offset); // TODO: PRESERVED IN REBASE + JsonArray hdfsOffsetMapToJSON(); // TODO: PRESERVED IN REBASE! + HdfsOffset getBeginningOffsets(); // TODO: NEW! This method returns the starting offsets for all available (aka. filtered) topic partitions. + HdfsOffset getInitialEndOffsets(); // TODO: NEW! This method returns the end offsets for all available (aka. filtered) topic partitions. -} \ No newline at end of file +} diff --git a/src/main/java/com/teragrep/pth_06/planner/HdfsQueryProcessor.java b/src/main/java/com/teragrep/pth_06/planner/HdfsQueryProcessor.java index 10a5e8c0..9f484b5f 100644 --- a/src/main/java/com/teragrep/pth_06/planner/HdfsQueryProcessor.java +++ b/src/main/java/com/teragrep/pth_06/planner/HdfsQueryProcessor.java @@ -1,3 +1,48 @@ +/* + * Teragrep Archive Datasource (pth_06) + * 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.pth_06.planner; import com.google.gson.JsonArray; @@ -17,6 +62,7 @@ // Class for processing hdfs queries. public class HdfsQueryProcessor implements HdfsQuery { + private final Logger LOGGER = LoggerFactory.getLogger(HdfsQueryProcessor.class); private LinkedList topicPartitionList; private final HdfsDBClient hdfsDBClient; @@ -30,8 +76,12 @@ public HdfsQueryProcessor(Config config) { try { HdfsConditionWalker parser = new HdfsConditionWalker(); topicsRegexString = parser.fromString(config.query); - } catch (Exception e) { - throw new RuntimeException("HdfsQueryProcessor problems when construction Query conditions query:" + config.query + " exception:" + e); + } + catch (Exception e) { + throw new RuntimeException( + "HdfsQueryProcessor problems when construction Query conditions query:" + config.query + + " exception:" + e + ); } } if (topicsRegexString == null) { @@ -44,7 +94,8 @@ public HdfsQueryProcessor(Config config) { config, topicsRegexString // topicsRegexString only searches for the given topic/topics (aka. directories). ); - } catch (Exception e) { + } + catch (Exception e) { throw new RuntimeException(e); } @@ -55,10 +106,11 @@ public HdfsQueryProcessor(Config config) { for (HdfsTopicPartitionOffsetMetadata r : topicPartitionList) { long partitionStart = r.endOffset; if (!hdfsOffsetMap.containsKey(r.topicPartition)) { - hdfsOffsetMap.put(r.topicPartition, partitionStart+1); - } else { + hdfsOffsetMap.put(r.topicPartition, partitionStart + 1); + } + else { if (hdfsOffsetMap.get(r.topicPartition) < partitionStart) { - hdfsOffsetMap.replace(r.topicPartition, partitionStart+1); + hdfsOffsetMap.replace(r.topicPartition, partitionStart + 1); } } } @@ -71,20 +123,35 @@ private void seekToResults() { try { topicPartitionList = hdfsDBClient.pullToPartitionList(); // queries the list of topic partitions based on the topic name condition filtering. LOGGER.debug("HdfsQueryProcessor.seekToResults>"); - } catch (IOException e) { + } + catch (IOException e) { throw new RuntimeException(e); } } // Returns all the available HDFS file metadata between the given topic partition offsets. @Override - public LinkedList processBetweenHdfsFileMetadata(HdfsOffset startOffset, HdfsOffset endOffset) { + public LinkedList processBetweenHdfsFileMetadata( + HdfsOffset startOffset, + HdfsOffset endOffset + ) { LinkedList rv = new LinkedList<>(); Map endOffsetMap = endOffset.getOffsetMap(); Map startOffsetMap = startOffset.getOffsetMap(); for (HdfsTopicPartitionOffsetMetadata r : topicPartitionList) { - if ( (endOffsetMap.get(r.topicPartition) >= r.endOffset) & (startOffsetMap.get(r.topicPartition) <= r.endOffset) ) { - rv.add(new HdfsTopicPartitionOffsetMetadata(r.topicPartition, r.endOffset, r.hdfsFilePath, r.hdfsFileSize)); + if ( + (endOffsetMap.get(r.topicPartition) >= r.endOffset) + & (startOffsetMap.get(r.topicPartition) <= r.endOffset) + ) { + rv + .add( + new HdfsTopicPartitionOffsetMetadata( + r.topicPartition, + r.endOffset, + r.hdfsFilePath, + r.hdfsFileSize + ) + ); } } return rv; @@ -98,7 +165,15 @@ public void commit(HdfsOffset offset) { // Generate new topicPartitionList where the metadata with offset values lower than the offset values given as input parameter are filtered out. for (HdfsTopicPartitionOffsetMetadata r : topicPartitionList) { if (offsetMap.get(r.topicPartition) < r.endOffset) { - newTopicPartitionList.add(new HdfsTopicPartitionOffsetMetadata(r.topicPartition, r.endOffset, r.hdfsFilePath, r.hdfsFileSize)); + newTopicPartitionList + .add( + new HdfsTopicPartitionOffsetMetadata( + r.topicPartition, + r.endOffset, + r.hdfsFilePath, + r.hdfsFileSize + ) + ); } } topicPartitionList = newTopicPartitionList; @@ -132,7 +207,8 @@ public HdfsOffset getBeginningOffsets() { long partitionOffset = r.endOffset; if (!startOffset.containsKey(r.topicPartition)) { startOffset.put(r.topicPartition, partitionOffset); - } else { + } + else { if (startOffset.get(r.topicPartition) > partitionOffset) { startOffset.replace(r.topicPartition, partitionOffset); } @@ -152,7 +228,8 @@ public HdfsOffset getInitialEndOffsets() { // When going through the result, store the topic partition with the highest offset to the endOffset object. if (!endOffset.containsKey(r.topicPartition)) { endOffset.put(r.topicPartition, partitionOffset); - } else { + } + else { if (endOffset.get(r.topicPartition) < partitionOffset) { endOffset.replace(r.topicPartition, partitionOffset); } @@ -160,4 +237,4 @@ public HdfsOffset getInitialEndOffsets() { } return new HdfsOffset(endOffset); } -} \ No newline at end of file +} diff --git a/src/main/java/com/teragrep/pth_06/planner/KafkaQueryProcessor.java b/src/main/java/com/teragrep/pth_06/planner/KafkaQueryProcessor.java index a3a1f1ba..4e2d5c9f 100644 --- a/src/main/java/com/teragrep/pth_06/planner/KafkaQueryProcessor.java +++ b/src/main/java/com/teragrep/pth_06/planner/KafkaQueryProcessor.java @@ -272,8 +272,11 @@ public void seekToHdfsOffsets(JsonArray startOffsets) { // Deserialize JsonArray back to Map: Map hdfsStartOffsets = new HashMap<>(); for (JsonElement pa : startOffsets) { - JsonObject offsetObject = pa.getAsJsonObject(); - TopicPartition topicPartition = new TopicPartition(offsetObject.get("topic").getAsString(), offsetObject.get("partition").getAsInt()); + JsonObject offsetObject = pa.getAsJsonObject(); + TopicPartition topicPartition = new TopicPartition( + offsetObject.get("topic").getAsString(), + offsetObject.get("partition").getAsInt() + ); Long offset = offsetObject.get("offset").getAsLong(); hdfsStartOffsets.put(topicPartition, offset); } diff --git a/src/main/java/com/teragrep/pth_06/planner/offset/HdfsOffset.java b/src/main/java/com/teragrep/pth_06/planner/offset/HdfsOffset.java index 5a8e73c6..e7b41425 100644 --- a/src/main/java/com/teragrep/pth_06/planner/offset/HdfsOffset.java +++ b/src/main/java/com/teragrep/pth_06/planner/offset/HdfsOffset.java @@ -1,3 +1,48 @@ +/* + * Teragrep Archive Datasource (pth_06) + * 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.pth_06.planner.offset; import com.google.common.reflect.TypeToken; @@ -17,7 +62,8 @@ public class HdfsOffset extends Offset implements Serializable { // TODO: Implement everything that is needed for tracking the offsets for HDFS datasource. - private static final Type mapType = new TypeToken>() {}.getType(); + private static final Type mapType = new TypeToken>() { + }.getType(); private final Map serializedHdfsOffset; public HdfsOffset(Map offset) { @@ -33,6 +79,7 @@ public HdfsOffset(String s) { Gson gson = new Gson(); serializedHdfsOffset = gson.fromJson(s, mapType); } + public Map getOffsetMap() { Map rv = new HashMap<>(serializedHdfsOffset.size()); @@ -54,10 +101,9 @@ public String json() { Gson gson = new Gson(); return gson.toJson(serializedHdfsOffset); } + @Override public String toString() { - return "HdfsOffset{" + - "serializedHdfsOffset=" + serializedHdfsOffset + - '}'; + return "HdfsOffset{" + "serializedHdfsOffset=" + serializedHdfsOffset + '}'; } -} \ No newline at end of file +} diff --git a/src/main/java/com/teragrep/pth_06/planner/offset/SerializedDatasourceOffset.java b/src/main/java/com/teragrep/pth_06/planner/offset/SerializedDatasourceOffset.java index 48be8f57..68909e15 100644 --- a/src/main/java/com/teragrep/pth_06/planner/offset/SerializedDatasourceOffset.java +++ b/src/main/java/com/teragrep/pth_06/planner/offset/SerializedDatasourceOffset.java @@ -71,28 +71,19 @@ public SerializedDatasourceOffset(HdfsOffset hdfsOffset, LongOffset archiveOffse this.kafkaOffset = kafkaOffset; } - public SerializedDatasourceOffset( - LongOffset archiveOffset, - KafkaOffset kafkaOffset - ) { + public SerializedDatasourceOffset(LongOffset archiveOffset, KafkaOffset kafkaOffset) { this.hdfsOffset = null; this.archiveOffset = archiveOffset; this.kafkaOffset = kafkaOffset; } - public SerializedDatasourceOffset( - HdfsOffset hdfsOffset, - LongOffset archiveOffset - ) { + public SerializedDatasourceOffset(HdfsOffset hdfsOffset, LongOffset archiveOffset) { this.hdfsOffset = hdfsOffset; this.archiveOffset = archiveOffset; this.kafkaOffset = null; } - public SerializedDatasourceOffset( - HdfsOffset hdfsOffset, - KafkaOffset kafkaOffset - ) { + public SerializedDatasourceOffset(HdfsOffset hdfsOffset, KafkaOffset kafkaOffset) { this.hdfsOffset = hdfsOffset; this.archiveOffset = null; this.kafkaOffset = kafkaOffset; @@ -118,7 +109,7 @@ public SerializedDatasourceOffset(KafkaOffset kafkaOffset) { @Override public String toString() { - return "SerializedDatasourceOffset{" + "version=" + version + ", hdfsOffset" + hdfsOffset + ", archiveOffset=" + archiveOffset - + ", kafkaOffset=" + kafkaOffset + '}'; + return "SerializedDatasourceOffset{" + "version=" + version + ", hdfsOffset" + hdfsOffset + ", archiveOffset=" + + archiveOffset + ", kafkaOffset=" + kafkaOffset + '}'; } } diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/HdfsConditionWalker.java b/src/main/java/com/teragrep/pth_06/planner/walker/HdfsConditionWalker.java index dc0ae39a..8cc40e0f 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/HdfsConditionWalker.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/HdfsConditionWalker.java @@ -1,10 +1,55 @@ +/* + * Teragrep Archive Datasource (pth_06) + * 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.pth_06.planner.walker; import com.teragrep.jue_01.GlobToRegEx; import org.w3c.dom.Element; // HDFS planner walker, only aims to filter out the topics as the planner only outputs the metadata for AVRO-files containing the records. The rest of the condition handling is done in the separate tasker walker. -public class HdfsConditionWalker extends XmlWalker{ +public class HdfsConditionWalker extends XmlWalker { @Override String emitElem(Element current) { @@ -47,4 +92,4 @@ String emitUnaryOperation(String op, Element current) throws Exception { // NOT is a filter, not a topic matcher return null; } -} \ No newline at end of file +} diff --git a/src/main/java/com/teragrep/pth_06/scheduler/Batch.java b/src/main/java/com/teragrep/pth_06/scheduler/Batch.java index 2665b2f3..9f644d7d 100644 --- a/src/main/java/com/teragrep/pth_06/scheduler/Batch.java +++ b/src/main/java/com/teragrep/pth_06/scheduler/Batch.java @@ -96,7 +96,8 @@ public Batch processRange(Offset start, Offset end) { if (config.isArchiveEnabled) { if (slice == null) { slice = new ArchiveBatchSliceCollection(this.archiveQuery).processRange(start, end); - } else { + } + else { slice.addAll(new ArchiveBatchSliceCollection(this.archiveQuery).processRange(start, end)); } } diff --git a/src/main/java/com/teragrep/pth_06/scheduler/BatchSlice.java b/src/main/java/com/teragrep/pth_06/scheduler/BatchSlice.java index 7271722b..707c2995 100644 --- a/src/main/java/com/teragrep/pth_06/scheduler/BatchSlice.java +++ b/src/main/java/com/teragrep/pth_06/scheduler/BatchSlice.java @@ -111,7 +111,8 @@ public long getSize() { // The files in HDFS have a default size limiter set to them during creation. Use that limit (64000000) or hdfsTopicPartitionOffsetMetadata.getFileSize(). if (hdfsTopicPartitionOffsetMetadata != null) { return hdfsTopicPartitionOffsetMetadata.hdfsFileSize; - } else { + } + else { throw new RuntimeException("Expected HDFS file metadata, instead was null"); } default: @@ -122,6 +123,7 @@ public long getSize() { @Override public String toString() { return "BatchSlice{" + "batchSliceType=" + type + ", archiveS3ObjectMetadata=" + archiveS3ObjectMetadata - + ", kafkaTopicPartitionOffsetMetadata=" + kafkaTopicPartitionOffsetMetadata + ", hdfsTopicPartitionOffsetMetadata=" + hdfsTopicPartitionOffsetMetadata + '}'; + + ", kafkaTopicPartitionOffsetMetadata=" + kafkaTopicPartitionOffsetMetadata + + ", hdfsTopicPartitionOffsetMetadata=" + hdfsTopicPartitionOffsetMetadata + '}'; } } diff --git a/src/main/java/com/teragrep/pth_06/scheduler/HdfsBatchSliceCollection.java b/src/main/java/com/teragrep/pth_06/scheduler/HdfsBatchSliceCollection.java index 5ae1ebc0..f5cbf262 100644 --- a/src/main/java/com/teragrep/pth_06/scheduler/HdfsBatchSliceCollection.java +++ b/src/main/java/com/teragrep/pth_06/scheduler/HdfsBatchSliceCollection.java @@ -1,8 +1,52 @@ +/* + * Teragrep Archive Datasource (pth_06) + * 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.pth_06.scheduler; import com.teragrep.pth_06.HdfsTopicPartitionOffsetMetadata; import com.teragrep.pth_06.planner.HdfsQuery; -import com.teragrep.pth_06.planner.KafkaQuery; import com.teragrep.pth_06.planner.offset.DatasourceOffset; import com.teragrep.pth_06.planner.offset.HdfsOffset; import org.slf4j.Logger; @@ -12,10 +56,11 @@ import java.util.LinkedList; public final class HdfsBatchSliceCollection extends BatchSliceCollection { + private final Logger LOGGER = LoggerFactory.getLogger(HdfsBatchSliceCollection.class); private final HdfsQuery hq; - public HdfsBatchSliceCollection(HdfsQuery hq) { + public HdfsBatchSliceCollection(HdfsQuery hq) { super(); this.hq = hq; } @@ -28,7 +73,7 @@ public HdfsBatchSliceCollection processRange(Offset start, Offset end) { KafkaBatchSliceCollection rv = generate(kafkaStartOffset, kafkaEndOffset); LOGGER.debug("processRange(): arg start " + start + " arg end: " + end + " rv: " + rv ); return rv; - + ARCHIVE: LOGGER.debug("processRange(): args: start: " + start + " end: " + end); this.clear(); // clear internal list @@ -57,15 +102,14 @@ public HdfsBatchSliceCollection processRange(Offset start, Offset end) { // HDFS: // If the slices are not distributed correctly, refactor the code to use the archive approach instead of kafka approach. LOGGER.debug("processRange(): args: start: " + start + " end: " + end); - HdfsOffset hdfsStartOffset = ((DatasourceOffset)start).getHdfsOffset(); - HdfsOffset hdfsEndOffset = ((DatasourceOffset)end).getHdfsOffset(); - LinkedList result = hq.processBetweenHdfsFileMetadata(hdfsStartOffset, hdfsEndOffset); + HdfsOffset hdfsStartOffset = ((DatasourceOffset) start).getHdfsOffset(); + HdfsOffset hdfsEndOffset = ((DatasourceOffset) end).getHdfsOffset(); + LinkedList result = hq + .processBetweenHdfsFileMetadata(hdfsStartOffset, hdfsEndOffset); for (HdfsTopicPartitionOffsetMetadata r : result) { - this.add( - new BatchSlice(r) - ); + this.add(new BatchSlice(r)); } return this; } -} \ No newline at end of file +} diff --git a/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartition.java b/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartition.java index 16980db3..83f34b1f 100644 --- a/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartition.java +++ b/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartition.java @@ -1,3 +1,48 @@ +/* + * Teragrep Archive Datasource (pth_06) + * 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.pth_06.task; public class HdfsMicroBatchInputPartition { diff --git a/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReader.java b/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReader.java index f1eef110..adb5634d 100644 --- a/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReader.java +++ b/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReader.java @@ -1,3 +1,48 @@ +/* + * Teragrep Archive Datasource (pth_06) + * 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.pth_06.task; public class HdfsMicroBatchInputPartitionReader { diff --git a/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReader.java b/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReader.java index 6808aa6a..aa73272b 100644 --- a/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReader.java +++ b/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReader.java @@ -1,3 +1,48 @@ +/* + * Teragrep Archive Datasource (pth_06) + * 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.pth_06.task.hdfs; public class AvroReader { diff --git a/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConverter.java b/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConverter.java index afd23590..406740eb 100644 --- a/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConverter.java +++ b/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConverter.java @@ -1,3 +1,48 @@ +/* + * Teragrep Archive Datasource (pth_06) + * 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.pth_06.task.hdfs; public class HdfsRecordConverter { diff --git a/src/test/java/com/teragrep/pth_06/MockTeragrepDatasource.java b/src/test/java/com/teragrep/pth_06/MockTeragrepDatasource.java index be310025..60dccd23 100644 --- a/src/test/java/com/teragrep/pth_06/MockTeragrepDatasource.java +++ b/src/test/java/com/teragrep/pth_06/MockTeragrepDatasource.java @@ -129,7 +129,8 @@ public MicroBatchStream toMicroBatchStream(String checkpointLocation) { HdfsQuery hdfsQueryProcessor; if (config.isHdfsEnabled) { hdfsQueryProcessor = new HdfsQueryProcessor(config); - } else { + } + else { hdfsQueryProcessor = null; } @@ -147,7 +148,12 @@ public MicroBatchStream toMicroBatchStream(String checkpointLocation) { kafkaQueryProcessor = null; } - return new ArchiveMicroStreamReader(hdfsQueryProcessor, archiveQueryProcessor, kafkaQueryProcessor, config); + return new ArchiveMicroStreamReader( + hdfsQueryProcessor, + archiveQueryProcessor, + kafkaQueryProcessor, + config + ); } }; } diff --git a/src/test/java/com/teragrep/pth_06/planner/MockHDFS.java b/src/test/java/com/teragrep/pth_06/planner/MockHDFS.java index 58ee97b4..840533cf 100644 --- a/src/test/java/com/teragrep/pth_06/planner/MockHDFS.java +++ b/src/test/java/com/teragrep/pth_06/planner/MockHDFS.java @@ -1,3 +1,48 @@ +/* + * Teragrep Archive Datasource (pth_06) + * 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.pth_06.planner; import com.google.gson.JsonArray; @@ -41,7 +86,7 @@ public String startMiniCluster() throws IOException, InterruptedException { conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, baseDir.getAbsolutePath()); MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf); hdfsCluster = builder.build(); - hdfsURI = "hdfs://localhost:"+ hdfsCluster.getNameNodePort() + "/"; + hdfsURI = "hdfs://localhost:" + hdfsCluster.getNameNodePort() + "/"; // System.out.println("hdfsURI: " + hdfsURI); DistributedFileSystem fileSystem = hdfsCluster.getFileSystem(); @@ -57,7 +102,7 @@ public void teardownMiniCluster() { } public static void insertMockFiles() throws IOException { - String path = hdfsPath + "testConsumerTopic" ; // "hdfs:///opt/teragrep/cfe_39/srv/testConsumerTopic" + String path = hdfsPath + "testConsumerTopic"; // "hdfs:///opt/teragrep/cfe_39/srv/testConsumerTopic" // ====== Init HDFS File System Object Configuration conf = new Configuration(); // Set FileSystem URI @@ -71,7 +116,6 @@ public static 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. @@ -82,8 +126,9 @@ public static void insertMockFiles() throws IOException { LOGGER.debug("Path {} created.", path); } - String dir = System.getProperty("user.dir")+"/src/test/java/com/teragrep/pth_06/mockHdfsFiles"; - Set listOfFiles = Stream.of(Objects.requireNonNull(new File(dir).listFiles())) + String dir = System.getProperty("user.dir") + "/src/test/java/com/teragrep/pth_06/mockHdfsFiles"; + Set listOfFiles = Stream + .of(Objects.requireNonNull(new File(dir).listFiles())) .filter(file -> !file.isDirectory()) .map(File::getName) .collect(Collectors.toSet()); @@ -114,7 +159,8 @@ public static void insertMockFiles() throws IOException { public void simpleTest() { try { hdfsReadTest(); - } catch (IOException e) { + } + catch (IOException e) { throw new RuntimeException(e); } } @@ -137,9 +183,9 @@ public void hdfsReadTest() 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); @@ -159,15 +205,22 @@ public void hdfsReadTest() throws IOException { LOGGER.info("File belongs to partition: {}", partition); LOGGER.info("File has an offset of: {}", offset); if (!hdfsStartOffsets.containsKey(new TopicPartition(topic, Integer.parseInt(partition)))) { - hdfsStartOffsets.put(new TopicPartition(topic, Integer.parseInt(partition)), Long.parseLong(offset)); - } else { - if (hdfsStartOffsets.get(new TopicPartition(topic, Integer.parseInt(partition))) < Long.parseLong(offset)) { - hdfsStartOffsets.replace(new TopicPartition(topic, Integer.parseInt(partition)), Long.parseLong(offset)); + hdfsStartOffsets + .put(new TopicPartition(topic, Integer.parseInt(partition)), Long.parseLong(offset)); + } + else { + if ( + hdfsStartOffsets.get(new TopicPartition(topic, Integer.parseInt(partition))) < Long + .parseLong(offset) + ) { + hdfsStartOffsets + .replace(new TopicPartition(topic, Integer.parseInt(partition)), Long.parseLong(offset)); } } } } - }else { + } + else { LOGGER.info("No matching directories found"); } LOGGER.info("hdfsStartOffsets.toString(): "); @@ -180,10 +233,14 @@ public void hdfsReadTest() throws IOException { String topic = entry.getKey().topic(); String partition = String.valueOf(entry.getKey().partition()); String offset = String.valueOf(entry.getValue()); - rv.add(String.format( - "{\"topic\":\"%s\", \"partition\":\"%s\", \"offset\":\"%s\"}", - topic, partition, offset - )); + rv + .add( + String + .format( + "{\"topic\":\"%s\", \"partition\":\"%s\", \"offset\":\"%s\"}", topic, + partition, offset + ) + ); JsonObject jo = new JsonObject(); // TODO: Use this instead of string jo.addProperty("topic", topic); jo.addProperty("partition", partition); @@ -197,8 +254,11 @@ public void hdfsReadTest() throws IOException { // Deserialize ja back to Map Map offsetMap = new HashMap<>(); for (JsonElement pa : ja) { - JsonObject offsetObject = pa.getAsJsonObject(); - TopicPartition topicPartition = new TopicPartition(offsetObject.get("topic").getAsString(), offsetObject.get("partition").getAsInt()); + JsonObject offsetObject = pa.getAsJsonObject(); + TopicPartition topicPartition = new TopicPartition( + offsetObject.get("topic").getAsString(), + offsetObject.get("partition").getAsInt() + ); Long offset = offsetObject.get("offset").getAsLong(); offsetMap.put(topicPartition, offset); } @@ -214,6 +274,7 @@ public void hdfsReadTest() throws IOException { } private static final PathFilter pathFilter = new PathFilter() { + @Override public boolean accept(Path path) { return path.getName().matches("^testConsumer.*$"); // Catches the directory names. From 212a6a0e9564a9fac8761f2f58f9fc736c60e3ea Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Fri, 13 Sep 2024 16:14:17 +0300 Subject: [PATCH 03/62] Implemented incrementAndGetLatestOffset() method to HdfsQuery interface and HdfsQueryProcessor class for incremental batch slice creation. --- .../pth_06/ArchiveMicroStreamReader.java | 8 ++--- .../teragrep/pth_06/planner/HdfsQuery.java | 15 ++++---- .../pth_06/planner/HdfsQueryProcessor.java | 34 ++++++++++++++++++- 3 files changed, 44 insertions(+), 13 deletions(-) diff --git a/src/main/java/com/teragrep/pth_06/ArchiveMicroStreamReader.java b/src/main/java/com/teragrep/pth_06/ArchiveMicroStreamReader.java index 4afd75b8..04d4467d 100644 --- a/src/main/java/com/teragrep/pth_06/ArchiveMicroStreamReader.java +++ b/src/main/java/com/teragrep/pth_06/ArchiveMicroStreamReader.java @@ -275,7 +275,7 @@ else if (!this.config.isArchiveEnabled && this.config.isKafkaEnabled && !this.co } else if (!this.config.isArchiveEnabled && !this.config.isKafkaEnabled && this.config.isHdfsEnabled) { // only hdfs - rv = new DatasourceOffset(new HdfsOffset(this.hq.getInitialEndOffsets().getOffsetMap())); + rv = new DatasourceOffset(new HdfsOffset(this.hq.incrementAndGetLatestOffset().getOffsetMap())); } else if (this.config.isArchiveEnabled && this.config.isKafkaEnabled && !this.config.isHdfsEnabled) { // kafka and archive @@ -287,21 +287,21 @@ else if (this.config.isArchiveEnabled && this.config.isKafkaEnabled && !this.con else if (this.config.isArchiveEnabled && !this.config.isKafkaEnabled && this.config.isHdfsEnabled) { // archive and hdfs rv = new DatasourceOffset( - new HdfsOffset(this.hq.getInitialEndOffsets().getOffsetMap()), + new HdfsOffset(this.hq.incrementAndGetLatestOffset().getOffsetMap()), new LongOffset(this.aq.incrementAndGetLatestOffset()) ); } else if (!this.config.isArchiveEnabled && this.config.isKafkaEnabled && this.config.isHdfsEnabled) { // Kafka and HDFS rv = new DatasourceOffset( - new HdfsOffset(this.hq.getInitialEndOffsets().getOffsetMap()), + new HdfsOffset(this.hq.incrementAndGetLatestOffset().getOffsetMap()), new KafkaOffset(this.kq.getInitialEndOffsets()) ); } else if (this.config.isArchiveEnabled && this.config.isKafkaEnabled && this.config.isHdfsEnabled) { // all three rv = new DatasourceOffset( - new HdfsOffset(this.hq.getInitialEndOffsets().getOffsetMap()), + new HdfsOffset(this.hq.incrementAndGetLatestOffset().getOffsetMap()), new LongOffset(this.aq.incrementAndGetLatestOffset()), new KafkaOffset(this.kq.getInitialEndOffsets()) ); diff --git a/src/main/java/com/teragrep/pth_06/planner/HdfsQuery.java b/src/main/java/com/teragrep/pth_06/planner/HdfsQuery.java index 93fb3648..711d4d8f 100644 --- a/src/main/java/com/teragrep/pth_06/planner/HdfsQuery.java +++ b/src/main/java/com/teragrep/pth_06/planner/HdfsQuery.java @@ -51,21 +51,20 @@ import java.util.LinkedList; -// TODO: Make an interface for for HDFS query. methods are placeholders at the moment. - -// TODO: Fetches semi-recent data from HDFS. public interface HdfsQuery { LinkedList processBetweenHdfsFileMetadata( HdfsOffset startOffset, HdfsOffset endOffset - ); // TODO: PRESERVED IN REBASE, maybe rename it while at it. + ); + + void commit(HdfsOffset offset); - void commit(HdfsOffset offset); // TODO: PRESERVED IN REBASE + JsonArray hdfsOffsetMapToJSON(); - JsonArray hdfsOffsetMapToJSON(); // TODO: PRESERVED IN REBASE! + HdfsOffset getBeginningOffsets(); - HdfsOffset getBeginningOffsets(); // TODO: NEW! This method returns the starting offsets for all available (aka. filtered) topic partitions. + HdfsOffset getInitialEndOffsets(); // Delete after testing incrementAndGetLatestOffset() functionality thoroughly. - HdfsOffset getInitialEndOffsets(); // TODO: NEW! This method returns the end offsets for all available (aka. filtered) topic partitions. + HdfsOffset incrementAndGetLatestOffset(); // replaces getInitialEndOffsets() } diff --git a/src/main/java/com/teragrep/pth_06/planner/HdfsQueryProcessor.java b/src/main/java/com/teragrep/pth_06/planner/HdfsQueryProcessor.java index 9f484b5f..f2288b6f 100644 --- a/src/main/java/com/teragrep/pth_06/planner/HdfsQueryProcessor.java +++ b/src/main/java/com/teragrep/pth_06/planner/HdfsQueryProcessor.java @@ -57,6 +57,7 @@ import java.io.IOException; import java.util.HashMap; +import java.util.Iterator; import java.util.LinkedList; import java.util.Map; @@ -67,9 +68,17 @@ public class HdfsQueryProcessor implements HdfsQuery { private LinkedList topicPartitionList; private final HdfsDBClient hdfsDBClient; private String topicsRegexString; - private Map hdfsOffsetMap; + private final Map hdfsOffsetMap; + private final Map latestHdfsOffsetMap; + private final long quantumLength; + private final long numPartitions; + private final long totalObjectCountLimit; public HdfsQueryProcessor(Config config) { + // get configs from config object + this.quantumLength = config.batchConfig.quantumLength; + this.numPartitions = config.batchConfig.numPartitions; + this.totalObjectCountLimit = config.batchConfig.totalObjectCountLimit; // Filter only topics using regex pattern topicsRegexString = null; if (config.query != null) { @@ -114,6 +123,7 @@ public HdfsQueryProcessor(Config config) { } } } + latestHdfsOffsetMap = new HashMap<>(); LOGGER.debug("HdfsQueryProcessor.HdfsQueryProcessor>"); } @@ -237,4 +247,26 @@ public HdfsOffset getInitialEndOffsets() { } return new HdfsOffset(endOffset); } + + // Increments the latest offset values and returns that incremented offsets. Works by pulling data from the topicPartitionList until weight limit is reached. + public HdfsOffset incrementAndGetLatestOffset() { + if (this.latestHdfsOffsetMap.isEmpty()) { + HdfsOffset beginningOffsets = getBeginningOffsets(); + this.latestHdfsOffsetMap.putAll(beginningOffsets.getOffsetMap()); + } + // Initialize the batchSizeLimit object to split the data into appropriate sized batches + BatchSizeLimit batchSizeLimit = new BatchSizeLimit(quantumLength * numPartitions, totalObjectCountLimit); + // Keep loading more offsets from topicPartitionList until the limit is reached + Iterator iterator = topicPartitionList.iterator(); + while (!batchSizeLimit.isOverLimit() && iterator.hasNext()) { + HdfsTopicPartitionOffsetMetadata r = iterator.next(); + // When going through the result, store the topic partition with the highest offset to the latestHdfsOffsetMap. + if (latestHdfsOffsetMap.get(r.topicPartition) < r.endOffset) { + latestHdfsOffsetMap.replace(r.topicPartition, r.endOffset); + batchSizeLimit.add(r.hdfsFileSize); + } + } + return new HdfsOffset(latestHdfsOffsetMap); + } + } From 2677f4ac12cc4b0af27301cb2ac89764016b8eb9 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Mon, 16 Sep 2024 12:27:41 +0300 Subject: [PATCH 04/62] Implemented Kerberized hdfs access for HdfsDBClient. Implemented required configuration parameters to HdfsConfig and InstantiationTest. --- .../teragrep/pth_06/config/HdfsConfig.java | 37 ++++++- .../teragrep/pth_06/planner/HdfsDBClient.java | 96 +++++++++---------- .../teragrep/pth_06/InstantiationTest.java | 12 ++- 3 files changed, 94 insertions(+), 51 deletions(-) diff --git a/src/main/java/com/teragrep/pth_06/config/HdfsConfig.java b/src/main/java/com/teragrep/pth_06/config/HdfsConfig.java index a1d1b4ec..f643c4ac 100644 --- a/src/main/java/com/teragrep/pth_06/config/HdfsConfig.java +++ b/src/main/java/com/teragrep/pth_06/config/HdfsConfig.java @@ -54,8 +54,21 @@ public final class HdfsConfig { public final long cutoffOffset; // Represents the cutoff epoch which dictates which files should not be fetched from HDFS based on their timestamps. public final String hdfsPath; // Represents the working directory path in HDFS filesystem. public final String hdfsUri; // Represents the address of the HDFS server. + public final String UseHdfsHostname; + public final String hdfsTransferProtection; + public final String hdfsCipherSuites; public final boolean useMockHdfsDatabase; // Represents the configuration parameter if mock database should be used or not. Used only for testing. + // Kerberos + public final String kerberosAuthentication; + public final String kerberosAuthorization; + public final String kerberosPrincipalPattern; + public final String kerberosKdc; + public final String kerberosRealm; + public final String kerberosKeytabUser; + public final String kerberosKeytabPath; + public final String kerberosAutorenewal; + public final boolean isStub; public HdfsConfig(Map opts) { @@ -69,8 +82,19 @@ public HdfsConfig(Map opts) { ); // Default is -72h from now hdfsPath = getOrThrow(opts, "hdfs.hdfsPath"); hdfsUri = getOrThrow(opts, "hdfs.hdfsUri"); - String useMockHdfsDatabaseString = opts.getOrDefault("useMockHdfsDatabase", "false"); + UseHdfsHostname = getOrThrow(opts, "hdfs.UseHostName"); + hdfsTransferProtection = getOrThrow(opts, "hdfs.transferProtection"); + hdfsCipherSuites = getOrThrow(opts, "hdfs.cipherSuites"); + String useMockHdfsDatabaseString = opts.getOrDefault("hdfs.useMockHdfsDatabase", "false"); useMockHdfsDatabase = "true".equals(useMockHdfsDatabaseString); + kerberosAuthentication = getOrThrow(opts, "hdfs.krbAuthentication"); + kerberosAuthorization = getOrThrow(opts, "hdfs.krbAuthorization"); + kerberosPrincipalPattern = getOrThrow(opts, "hdfs.krbPrincipalPattern"); + kerberosKdc = getOrThrow(opts, "hdfs.krbKdc"); + kerberosRealm = getOrThrow(opts, "hdfs.krbRealm"); + kerberosKeytabUser = getOrThrow(opts, "hdfs.krbKeytabUser"); + kerberosKeytabPath = getOrThrow(opts, "hdfs.krbKeytabPath"); + kerberosAutorenewal = opts.getOrDefault("hdfs.useKerberosAutorenewal", "false"); isStub = false; } @@ -78,7 +102,18 @@ public HdfsConfig() { cutoffOffset = 0L; hdfsPath = ""; hdfsUri = ""; + UseHdfsHostname = ""; + hdfsTransferProtection = ""; + hdfsCipherSuites = ""; useMockHdfsDatabase = false; + kerberosAuthentication = ""; + kerberosAuthorization = ""; + kerberosPrincipalPattern = ""; + kerberosKdc = ""; + kerberosRealm = ""; + kerberosKeytabUser = ""; + kerberosKeytabPath = ""; + kerberosAutorenewal = "false"; isStub = true; } diff --git a/src/main/java/com/teragrep/pth_06/planner/HdfsDBClient.java b/src/main/java/com/teragrep/pth_06/planner/HdfsDBClient.java index a45e091a..1be1fbe0 100644 --- a/src/main/java/com/teragrep/pth_06/planner/HdfsDBClient.java +++ b/src/main/java/com/teragrep/pth_06/planner/HdfsDBClient.java @@ -74,13 +74,54 @@ public HdfsDBClient(Config config, String topicsRegexString) throws IOException this.topicsRegexString = topicsRegexString; this.ignoreBeforeEpoch = config.hdfsConfig.cutoffOffset; // Defines the minimum time window / cutoff epoch for making sure that the files which metadata is fetched by the planner are not mistakenly deleted by cfe_39 pruning before tasker can process the records inside. // Implement HDFS FileSystem access here - if (config.hdfsConfig.useMockHdfsDatabase) { + if ("kerberos".equals(config.hdfsConfig.kerberosAuthentication)) { + // Code for initializing the class with kerberos. + hdfsuri = config.hdfsConfig.hdfsUri; // 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.hdfsConfig.hdfsPath; + + // set kerberos host and realm + System.setProperty("java.security.krb5.realm", config.hdfsConfig.kerberosRealm); + System.setProperty("java.security.krb5.kdc", config.hdfsConfig.kerberosKdc); + + conf = new Configuration(); + + // enable kerberus + conf.set("hadoop.security.authentication", config.hdfsConfig.kerberosAuthentication); + conf.set("hadoop.security.authorization", config.hdfsConfig.kerberosAuthorization); + conf.set("hadoop.kerberos.keytab.login.autorenewal.enabled", config.hdfsConfig.kerberosAutorenewal); + + 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.hdfsConfig.UseHdfsHostname); + + // server principal, the kerberos principle that the namenode is using + conf.set("dfs.namenode.kerberos.principal.pattern", config.hdfsConfig.kerberosPrincipalPattern); + + conf.set("dfs.data.transfer.protection", config.hdfsConfig.hdfsTransferProtection); + conf.set("dfs.encrypt.data.transfer.cipher.suites", config.hdfsConfig.hdfsCipherSuites); + + // set usergroup stuff + UserGroupInformation.setConfiguration(conf); + UserGroupInformation + .loginUserFromKeytab(config.hdfsConfig.kerberosKeytabUser, config.hdfsConfig.kerberosKeytabPath); + + // filesystem for HDFS access is set here + fs = FileSystem.get(conf); + } + else { // Code for initializing the class in test mode without kerberos. hdfsuri = config.hdfsConfig.hdfsUri; // 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. - // 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.hdfsConfig.hdfsPath; // Example path: hdfs://localhost:34509/opt/teragrep/cfe_39/srv/ // ====== Init HDFS File System Object @@ -100,50 +141,7 @@ public HdfsDBClient(Config config, String topicsRegexString) throws IOException catch (IOException e) { throw new RuntimeException(e); } - } - else { - // Code for initializing the class with kerberos. - hdfsuri = config.hdfsConfig.hdfsUri; // 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.hdfsConfig.hdfsPath; - - // 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? - // TODO: Add all the Kerberos parameters to Config - - // 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); } } @@ -152,8 +150,8 @@ public LinkedList pullToPartitionList() throws LinkedList rv = new LinkedList<>(); // path holds the fileSystem path to the directory that holds a collection of other directories, each different directory representing a different topic. FileStatus[] directoryStatuses = fs.listStatus(new Path(path), topicFilter); - // If the path only holds one directory, fileStatuses will only hold one FileStatus object which returns this value when fileStatus.getPath() is called: - // hdfs://localhost:34509/opt/teragrep/cfe_39/srv/testConsumerTopic + /*If the path only holds one directory, fileStatuses will only hold one FileStatus object which returns this value when fileStatus.getPath() is called: + hdfs://localhost:34509/opt/teragrep/cfe_39/srv/testConsumerTopic*/ // Get the directory statuses. Each directory represents a Kafka topic. if (directoryStatuses.length > 0) { diff --git a/src/test/java/com/teragrep/pth_06/InstantiationTest.java b/src/test/java/com/teragrep/pth_06/InstantiationTest.java index 2c13700b..94458369 100644 --- a/src/test/java/com/teragrep/pth_06/InstantiationTest.java +++ b/src/test/java/com/teragrep/pth_06/InstantiationTest.java @@ -162,8 +162,18 @@ public void fullScanTest() throws StreamingQueryException, TimeoutException { .option("hdfs.enabled", "true") .option("hdfs.hdfsPath", hdfsPath) .option("hdfs.hdfsUri", hdfsUri) + .option("hdfs.UseHostName", "false") + .option("hdfs.transferProtection", "test") + .option("hdfs.cipherSuites", "test") .option("hdfs.useMockHdfsDatabase", "true") - //.option("hdfs.hdfsCutoffOffset", Instant.now().toEpochMilli()) // Current time - pruning offset from cfe_39 + configurable cutoff offset. Granularity is in milliseconds. + .option("hdfs.krbAuthentication", "false") + .option("hdfs.krbAuthorization", "test") + .option("hdfs.krbPrincipalPattern", "test") + .option("hdfs.krbKdc", "test") + .option("hdfs.krbRealm", "test") + .option("hdfs.krbKeytabUser", "test") + .option("hdfs.krbKeytabPath", "test") + .option("hdfs.useKerberosAutorenewal", "true") .load(); Dataset df2 = df.agg(functions.count("*")); From d2337ab50525966f87a96d505e3bc724d1684505 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Tue, 17 Sep 2024 11:28:34 +0300 Subject: [PATCH 05/62] Added avro dependencies and schema. --- pom.xml | 32 ++++++++++++++++++++++++++++++++ src/main/avro/KafkaRecord.avsc | 15 +++++++++++++++ 2 files changed, 47 insertions(+) create mode 100644 src/main/avro/KafkaRecord.avsc diff --git a/pom.xml b/pom.xml index 04ae152e..e635ac5c 100644 --- a/pom.xml +++ b/pom.xml @@ -269,6 +269,17 @@ 2.1.0 compile + + + org.xerial.snappy + snappy-java + 1.1.10.5 + + + org.apache.avro + avro + 1.11.3 + ${project.basedir}/target @@ -340,6 +351,9 @@ README.adoc README.md + + src/main/avro/KafkaRecord.avsc + src/main/java/com/teragrep/pth_06/avro/SyslogRecord.java @@ -351,6 +365,23 @@ + + org.apache.avro + avro-maven-plugin + 1.11.3 + + + + schema + + generate-sources + + ${project.basedir}/src/main/avro/ + ${project.basedir}/src/main/java/ + + + + org.jooq jooq-codegen-maven @@ -404,6 +435,7 @@ src/main/java/com/teragrep/pth_06/jooq/generated/**/*.* + src/main/java/com/teragrep/pth_06/avro/SyslogRecord.java diff --git a/src/main/avro/KafkaRecord.avsc b/src/main/avro/KafkaRecord.avsc new file mode 100644 index 00000000..78406df7 --- /dev/null +++ b/src/main/avro/KafkaRecord.avsc @@ -0,0 +1,15 @@ +{"namespace": "com.teragrep.pth_06.avro", + "type": "record", + "name": "SyslogRecord", + "fields": [ + {"name": "timestamp", "type": "long"}, + {"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": "payload", "type": "string"} + ] +} \ No newline at end of file From b2ee2558ff69a7f25aa990c09e7531d148bae600 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Tue, 17 Sep 2024 11:33:45 +0300 Subject: [PATCH 06/62] Updated mock hdfs files with files built using the updated schema. --- .../teragrep/pth_06/InstantiationTest.java | 2 +- .../com/teragrep/pth_06/mockHdfsFiles/0.13 | Bin 1982 -> 1621 bytes .../com/teragrep/pth_06/mockHdfsFiles/0.8 | Bin 2692 -> 0 bytes .../com/teragrep/pth_06/mockHdfsFiles/0.9 | Bin 0 -> 2861 bytes 4 files changed, 1 insertion(+), 1 deletion(-) delete mode 100644 src/test/java/com/teragrep/pth_06/mockHdfsFiles/0.8 create mode 100644 src/test/java/com/teragrep/pth_06/mockHdfsFiles/0.9 diff --git a/src/test/java/com/teragrep/pth_06/InstantiationTest.java b/src/test/java/com/teragrep/pth_06/InstantiationTest.java index 94458369..fbac2a74 100644 --- a/src/test/java/com/teragrep/pth_06/InstantiationTest.java +++ b/src/test/java/com/teragrep/pth_06/InstantiationTest.java @@ -123,7 +123,7 @@ public void prepareEnv() throws Exception { //spark.sparkContext().setLogLevel("ERROR"); - expectedRows = preloadS3Data() + MockKafkaConsumerFactory.getNumRecords() - 9; + expectedRows = preloadS3Data() + MockKafkaConsumerFactory.getNumRecords() - 10; /* Remove the - modifier once the HDFS tasker is up and running. If file 0.8 is stored to HDFS, 9 records out of 14 will be read from HDFS instead of Kafka. If files 0.8 and 0.13 are stored to HDFS, all 14 records will be read from HDFS instead of Kafka.*/ diff --git a/src/test/java/com/teragrep/pth_06/mockHdfsFiles/0.13 b/src/test/java/com/teragrep/pth_06/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/pth_06/mockHdfsFiles/0.8 b/src/test/java/com/teragrep/pth_06/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 e90e57c2b4d75497d971b84a137f27a45e14076b Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Tue, 17 Sep 2024 13:42:19 +0300 Subject: [PATCH 07/62] Implemented preliminary code for tasker component. S3-HDFS cutoff filtering WIP. --- .../pth_06/ArchiveMicroStreamReader.java | 17 +-- .../teragrep/pth_06/planner/HdfsDBClient.java | 7 +- .../task/HdfsMicroBatchInputPartition.java | 53 ++++++- .../HdfsMicroBatchInputPartitionReader.java | 142 +++++++++++++++++- .../task/TeragrepPartitionReaderFactory.java | 26 ++++ .../teragrep/pth_06/task/hdfs/AvroReader.java | 60 ++++++++ .../pth_06/task/hdfs/HdfsRecordConverter.java | 54 +++++++ .../teragrep/pth_06/InstantiationTest.java | 7 +- 8 files changed, 344 insertions(+), 22 deletions(-) diff --git a/src/main/java/com/teragrep/pth_06/ArchiveMicroStreamReader.java b/src/main/java/com/teragrep/pth_06/ArchiveMicroStreamReader.java index 04d4467d..a6c3c09b 100644 --- a/src/main/java/com/teragrep/pth_06/ArchiveMicroStreamReader.java +++ b/src/main/java/com/teragrep/pth_06/ArchiveMicroStreamReader.java @@ -54,6 +54,7 @@ import com.teragrep.pth_06.planner.offset.KafkaOffset; import com.teragrep.pth_06.scheduler.*; import com.teragrep.pth_06.task.ArchiveMicroBatchInputPartition; +import com.teragrep.pth_06.task.HdfsMicroBatchInputPartition; import com.teragrep.pth_06.task.TeragrepPartitionReaderFactory; import com.teragrep.pth_06.task.KafkaMicroBatchInputPartition; import org.apache.spark.sql.connector.read.InputPartition; @@ -364,16 +365,12 @@ public InputPartition[] planInputPartitions(Offset start, Offset end) { } if (!hdfsTaskList.isEmpty()) { - // BatchSliceType.HDFS contains the metadata for the HDFS files that contain the records that are being queried. Available topics in HDFS are already filtered based on the spark query conditions. - // The records that are inside the files are fetched and processed in the tasker. Tasker does rest of the filtering based on the given query conditions, for example the cutoff epoch handling between the records that are fetched from S3 and HDFS. - // The Spark planner/scheduler is only single-threaded while tasker is multithreaded. Planner is not suitable for fetching and processing all the records, it should be done in tasker which will handle the processing in multithreaded environment based on batch slices. - - // Implement HdfsMicroBatchInputPartition usage here. If needed change the slice creation to be incremental like it is with archive, for that create HDFS variant of the incrementAndGetLatestOffset()-method from ArchiveQueryProcessor. - // At the moment it fetches all the metadata available at once and puts them into hdfsTaskList. - /*inputPartitions.add(new HdfsMicroBatchInputPartition( - config.hdfsConfig, - hdfsTaskList - ));*/ + /* BatchSliceType.HDFS contains the metadata for the HDFS files that contain the records that are being queried. Available topics in HDFS are already filtered based on the spark query conditions. + The records that are inside the files are fetched and processed in the tasker. Tasker does rest of the filtering based on the given query conditions, for example the cutoff epoch handling between the records that are fetched from S3 and HDFS. + The Spark planner/scheduler is only single-threaded while tasker is multithreaded. Planner is not suitable for fetching and processing all the records, it should be done in tasker which will handle the processing in multithreaded environment based on batch slices. + The slice creation has been changed to be incremental, which means that not all avro-files are fetched from hdfs at once. Instead, the files are distributed over several batches based on file size.*/ + + inputPartitions.add(new HdfsMicroBatchInputPartition(config.hdfsConfig, hdfsTaskList)); } // kafka tasks diff --git a/src/main/java/com/teragrep/pth_06/planner/HdfsDBClient.java b/src/main/java/com/teragrep/pth_06/planner/HdfsDBClient.java index 1be1fbe0..8358b6f9 100644 --- a/src/main/java/com/teragrep/pth_06/planner/HdfsDBClient.java +++ b/src/main/java/com/teragrep/pth_06/planner/HdfsDBClient.java @@ -135,12 +135,7 @@ These values should be fetched from config and other input parameters (topic+par 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); - } + fs = FileSystem.get(URI.create(hdfsuri), conf); } } diff --git a/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartition.java b/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartition.java index 83f34b1f..9f2de4ae 100644 --- a/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartition.java +++ b/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartition.java @@ -45,5 +45,56 @@ */ package com.teragrep.pth_06.task; -public class HdfsMicroBatchInputPartition { +import com.teragrep.pth_06.HdfsTopicPartitionOffsetMetadata; +import com.teragrep.pth_06.config.HdfsConfig; +import org.apache.spark.sql.connector.read.InputPartition; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.LinkedList; + +public class HdfsMicroBatchInputPartition implements InputPartition { + + public final Logger LOGGER = LoggerFactory.getLogger(HdfsMicroBatchInputPartition.class); + + public final LinkedList taskObjectList; + + public final long cutoffOffset; // Represents the cutoff epoch which dictates which files should not be fetched from HDFS based on their timestamps. + public final String hdfsPath; // Represents the working directory path in HDFS filesystem. + public final String hdfsUri; // Represents the address of the HDFS server. + public final String UseHdfsHostname; + public final String hdfsTransferProtection; + public final String hdfsCipherSuites; + public final boolean useMockHdfsDatabase; // Represents the configuration parameter if mock database should be used or not. Used only for testing. + public final String kerberosAuthentication; + public final String kerberosAuthorization; + public final String kerberosPrincipalPattern; + public final String kerberosKdc; + public final String kerberosRealm; + public final String kerberosKeytabUser; + public final String kerberosKeytabPath; + public final String kerberosAutorenewal; + + public HdfsMicroBatchInputPartition( + HdfsConfig hdfsConfig, + LinkedList hdfsTaskList + ) { + this.taskObjectList = hdfsTaskList; + kerberosAutorenewal = hdfsConfig.kerberosAutorenewal; + hdfsUri = hdfsConfig.hdfsUri; + hdfsPath = hdfsConfig.hdfsPath; + cutoffOffset = hdfsConfig.cutoffOffset; + UseHdfsHostname = hdfsConfig.UseHdfsHostname; + hdfsTransferProtection = hdfsConfig.hdfsTransferProtection; + hdfsCipherSuites = hdfsConfig.hdfsCipherSuites; + useMockHdfsDatabase = hdfsConfig.useMockHdfsDatabase; + kerberosAuthentication = hdfsConfig.kerberosAuthentication; + kerberosAuthorization = hdfsConfig.kerberosAuthorization; + kerberosPrincipalPattern = hdfsConfig.kerberosPrincipalPattern; + kerberosKdc = hdfsConfig.kerberosKdc; + kerberosRealm = hdfsConfig.kerberosRealm; + kerberosKeytabUser = hdfsConfig.kerberosKeytabUser; + kerberosKeytabPath = hdfsConfig.kerberosKeytabPath; + } + } diff --git a/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReader.java b/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReader.java index adb5634d..2795b0c1 100644 --- a/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReader.java +++ b/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReader.java @@ -45,5 +45,145 @@ */ package com.teragrep.pth_06.task; -public class HdfsMicroBatchInputPartitionReader { +import com.teragrep.pth_06.HdfsTopicPartitionOffsetMetadata; +import com.teragrep.pth_06.task.hdfs.HdfsRecordConverter; +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.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.connector.read.PartitionReader; + +import java.io.IOException; +import java.net.URI; +import java.util.LinkedList; + +public class HdfsMicroBatchInputPartitionReader implements PartitionReader { + + private final LinkedList taskObjectList; + private final FileSystem fs; + private HdfsRecordConverter hdfsRecordConverter; + + public HdfsMicroBatchInputPartitionReader( + String kerberosAuthentication, + String hdfsUri, + String hdfsPath, + String kerberosRealm, + String kerberosKdc, + String kerberosAuthorization, + String kerberosAutorenewal, + String UseHdfsHostname, + String kerberosPrincipalPattern, + String hdfsTransferProtection, + String hdfsCipherSuites, + String kerberosKeytabUser, + String kerberosKeytabPath, + LinkedList taskObjectList + ) throws IOException { + this.taskObjectList = taskObjectList; + + // FIXME: Implement FileSystem initialization code somewhere else than the constructor. + if ("kerberos".equals(kerberosAuthentication)) { + // Code for initializing the class with 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. + 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).*/ + + // set kerberos host and realm + System.setProperty("java.security.krb5.realm", kerberosRealm); + System.setProperty("java.security.krb5.kdc", kerberosKdc); + + Configuration conf = new Configuration(); + + // enable kerberus + conf.set("hadoop.security.authentication", kerberosAuthentication); + conf.set("hadoop.security.authorization", kerberosAuthorization); + conf.set("hadoop.kerberos.keytab.login.autorenewal.enabled", kerberosAutorenewal); + + 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", UseHdfsHostname); + + // server principal, the kerberos principle that the namenode is using + conf.set("dfs.namenode.kerberos.principal.pattern", kerberosPrincipalPattern); + + conf.set("dfs.data.transfer.protection", hdfsTransferProtection); + conf.set("dfs.encrypt.data.transfer.cipher.suites", hdfsCipherSuites); + + // set usergroup stuff + UserGroupInformation.setConfiguration(conf); + UserGroupInformation.loginUserFromKeytab(kerberosKeytabUser, kerberosKeytabPath); + + // filesystem for HDFS access is set here + fs = FileSystem.get(conf); + } + else { + // Code for initializing the class in test mode without 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. + 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).*/ + + // ====== 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 + fs = FileSystem.get(URI.create(hdfsUri), conf); + } + this.hdfsRecordConverter = new HdfsRecordConverter(fs); // stub + } + + // Read avro-file until it ends + @Override + public boolean next() throws IOException { + // true if data is available, false if not + boolean rv = false; + + while (!taskObjectList.isEmpty() && !rv) { + if (hdfsRecordConverter.isStub()) { + hdfsRecordConverter = new HdfsRecordConverter(fs, taskObjectList.getFirst()); + hdfsRecordConverter.open(); + } + + rv = hdfsRecordConverter.next(); + + if (!rv) { + // object was consumed + hdfsRecordConverter.close(); + + // remove consumed object + taskObjectList.removeFirst(); + if (!taskObjectList.isEmpty()) { + hdfsRecordConverter = new HdfsRecordConverter(fs, taskObjectList.getFirst()); + hdfsRecordConverter.open(); + } + } + + } + return rv; + } + + @Override + public InternalRow get() { + return hdfsRecordConverter.get(); + } + + @Override + public void close() throws IOException { + if (!hdfsRecordConverter.isStub()) { + hdfsRecordConverter.close(); + } + } } diff --git a/src/main/java/com/teragrep/pth_06/task/TeragrepPartitionReaderFactory.java b/src/main/java/com/teragrep/pth_06/task/TeragrepPartitionReaderFactory.java index 589ccf79..e168a0ec 100644 --- a/src/main/java/com/teragrep/pth_06/task/TeragrepPartitionReaderFactory.java +++ b/src/main/java/com/teragrep/pth_06/task/TeragrepPartitionReaderFactory.java @@ -50,6 +50,8 @@ import org.apache.spark.sql.connector.read.PartitionReader; import org.apache.spark.sql.connector.read.PartitionReaderFactory; +import java.io.IOException; + /** *

TeragrepPartitionReaderFactory

Used to create appropriate PartitionReaders based on the type of the * InputPartition provided. @@ -112,6 +114,30 @@ else if (inputPartition instanceof KafkaMicroBatchInputPartition) { kip.skipNonRFC5424Records ); } + else if (inputPartition instanceof HdfsMicroBatchInputPartition) { + HdfsMicroBatchInputPartition hip = (HdfsMicroBatchInputPartition) inputPartition; + try { + return new HdfsMicroBatchInputPartitionReader( + hip.kerberosAuthentication, + hip.hdfsUri, + hip.hdfsPath, + hip.kerberosRealm, + hip.kerberosKdc, + hip.kerberosAuthorization, + hip.kerberosAutorenewal, + hip.UseHdfsHostname, + hip.kerberosPrincipalPattern, + hip.hdfsTransferProtection, + hip.hdfsCipherSuites, + hip.kerberosKeytabUser, + hip.kerberosKeytabPath, + hip.taskObjectList + ); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } // inputPartition is neither Archive nor Kafka type throw new RuntimeException( diff --git a/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReader.java b/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReader.java index aa73272b..0b0b6094 100644 --- a/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReader.java +++ b/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReader.java @@ -45,5 +45,65 @@ */ package com.teragrep.pth_06.task.hdfs; +import com.teragrep.pth_06.HdfsTopicPartitionOffsetMetadata; +import com.teragrep.pth_06.avro.SyslogRecord; +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.Path; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter; +import org.apache.spark.unsafe.types.UTF8String; + +import java.io.IOException; + public class AvroReader { + + private final FileSystem fs; + private final HdfsTopicPartitionOffsetMetadata hdfsTopicPartitionOffsetMetadata; + private final UnsafeRowWriter rowWriter; + private FSDataInputStream inputStream; + private DataFileStream reader; + + // This class will allow reading the contents of the avro-files that are using SyslogRecord schema. + public AvroReader(FileSystem fs, HdfsTopicPartitionOffsetMetadata hdfsTopicPartitionOffsetMetadata) { + this.fs = fs; + this.hdfsTopicPartitionOffsetMetadata = hdfsTopicPartitionOffsetMetadata; + this.rowWriter = new UnsafeRowWriter(11); + } + + public void open() throws IOException { + Path hdfsreadpath = new Path(hdfsTopicPartitionOffsetMetadata.hdfsFilePath); + inputStream = fs.open(hdfsreadpath); + reader = new DataFileStream<>(inputStream, new SpecificDatumReader<>(SyslogRecord.class)); + } + + public boolean next() { + return reader.hasNext(); + } + + public InternalRow get() { + + SyslogRecord next = reader.next(); + + rowWriter.reset(); + rowWriter.zeroOutNullBytes(); + rowWriter.write(0, next.getTimestamp()); + rowWriter.write(1, UTF8String.fromString(next.getPayload().toString())); + rowWriter.write(2, UTF8String.fromString(next.getDirectory().toString())); + rowWriter.write(3, UTF8String.fromString(next.getStream().toString())); + rowWriter.write(4, UTF8String.fromString(next.getHost().toString())); + rowWriter.write(5, UTF8String.fromString(next.getInput().toString())); + rowWriter.write(6, UTF8String.fromString(next.getPartition().toString())); + rowWriter.write(7, UTF8String.fromString(String.valueOf(next.getOffset()))); + rowWriter.write(8, UTF8String.fromString(next.getOrigin().toString())); + + return rowWriter.getRow(); + } + + public void close() throws IOException { + reader.close(); + } + } diff --git a/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConverter.java b/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConverter.java index 406740eb..1772cb63 100644 --- a/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConverter.java +++ b/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConverter.java @@ -45,5 +45,59 @@ */ package com.teragrep.pth_06.task.hdfs; +import com.teragrep.pth_06.HdfsTopicPartitionOffsetMetadata; +import org.apache.hadoop.fs.FileSystem; +import org.apache.kafka.common.TopicPartition; +import org.apache.spark.sql.catalyst.InternalRow; + +import java.io.IOException; + +// This class will read the records from avro-files fetched from HDFS with the help of AvroReader and convert them to the schema/format used by pth_06. public class HdfsRecordConverter { + + private final FileSystem fs; + private final HdfsTopicPartitionOffsetMetadata hdfsTopicPartitionOffsetMetadata; + private final boolean stub; + private final AvroReader avroReader; + + // Stub object + public HdfsRecordConverter(FileSystem fs) { + this(fs, new HdfsTopicPartitionOffsetMetadata(new TopicPartition("", 0), 0, "", 0), true); + } + + public HdfsRecordConverter(FileSystem fs, HdfsTopicPartitionOffsetMetadata hdfsTopicPartitionOffsetMetadata) { + this(fs, hdfsTopicPartitionOffsetMetadata, false); + } + + public HdfsRecordConverter( + FileSystem fs, + HdfsTopicPartitionOffsetMetadata hdfsTopicPartitionOffsetMetadata, + Boolean stub + ) { + this.fs = fs; + this.hdfsTopicPartitionOffsetMetadata = hdfsTopicPartitionOffsetMetadata; + this.avroReader = new AvroReader(this.fs, this.hdfsTopicPartitionOffsetMetadata); + this.stub = stub; + } + + public void open() throws IOException { + avroReader.open(); + } + + public void close() throws IOException { + avroReader.close(); + } + + public boolean next() { + return avroReader.next(); + } + + public InternalRow get() { + return avroReader.get(); + } + + public boolean isStub() { + return stub; + } + } diff --git a/src/test/java/com/teragrep/pth_06/InstantiationTest.java b/src/test/java/com/teragrep/pth_06/InstantiationTest.java index fbac2a74..a5004833 100644 --- a/src/test/java/com/teragrep/pth_06/InstantiationTest.java +++ b/src/test/java/com/teragrep/pth_06/InstantiationTest.java @@ -123,10 +123,9 @@ public void prepareEnv() throws Exception { //spark.sparkContext().setLogLevel("ERROR"); - expectedRows = preloadS3Data() + MockKafkaConsumerFactory.getNumRecords() - 10; - /* Remove the - modifier once the HDFS tasker is up and running. - If file 0.8 is stored to HDFS, 9 records out of 14 will be read from HDFS instead of Kafka. - If files 0.8 and 0.13 are stored to HDFS, all 14 records will be read from HDFS instead of Kafka.*/ + expectedRows = preloadS3Data() + MockKafkaConsumerFactory.getNumRecords(); + /*If file 0.9 is stored to HDFS, 10 records out of 14 will be read from HDFS instead of Kafka. + If files 0.9 and 0.13 are stored to HDFS, all 14 records will be read from HDFS instead of Kafka.*/ } @Test From 66b420adc653d69d1b04ea49d84daeed0ab72388 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Wed, 18 Sep 2024 10:03:48 +0300 Subject: [PATCH 08/62] Implemented preliminary S3-HDFS cutoff filtering. --- .../teragrep/pth_06/config/HdfsConfig.java | 17 +++++++++--- .../teragrep/pth_06/planner/HdfsDBClient.java | 2 +- .../task/HdfsMicroBatchInputPartition.java | 4 +-- .../HdfsMicroBatchInputPartitionReader.java | 11 +++++++- .../task/TeragrepPartitionReaderFactory.java | 1 + .../teragrep/pth_06/task/hdfs/AvroReader.java | 27 ++++++++++--------- .../teragrep/pth_06/InstantiationTest.java | 1 + 7 files changed, 43 insertions(+), 20 deletions(-) diff --git a/src/main/java/com/teragrep/pth_06/config/HdfsConfig.java b/src/main/java/com/teragrep/pth_06/config/HdfsConfig.java index f643c4ac..25ef78b7 100644 --- a/src/main/java/com/teragrep/pth_06/config/HdfsConfig.java +++ b/src/main/java/com/teragrep/pth_06/config/HdfsConfig.java @@ -51,7 +51,8 @@ public final class HdfsConfig { // HDFS - public final long cutoffOffset; // Represents the cutoff epoch which dictates which files should not be fetched from HDFS based on their timestamps. + public final long includeFileEpochAndAfter; // Represents the cutoff epoch in milliseconds which dictates which files should not be fetched from HDFS based on their timestamps. + public final long includeRecordEpochAndAfter; // Represents the cutoff epoch in microseconds which dictates which record should not be fetched from HDFS based on the record timestamp. public final String hdfsPath; // Represents the working directory path in HDFS filesystem. public final String hdfsUri; // Represents the address of the HDFS server. public final String UseHdfsHostname; @@ -72,14 +73,21 @@ public final class HdfsConfig { public final boolean isStub; public HdfsConfig(Map opts) { - cutoffOffset = Long + includeFileEpochAndAfter = Long .parseLong( opts .getOrDefault( - "hdfs.hdfsCutoffOffset", + "hdfs.includeFileEpochAndAfter", String.valueOf(Instant.now().toEpochMilli() - 72 * 3600000) ) ); // Default is -72h from now + includeRecordEpochAndAfter = Long + .parseLong( + opts + .getOrDefault( + "hdfs.includeRecordEpochAndAfter", String.valueOf(Long.MIN_VALUE / (1000 * 1000)) + ) + ); hdfsPath = getOrThrow(opts, "hdfs.hdfsPath"); hdfsUri = getOrThrow(opts, "hdfs.hdfsUri"); UseHdfsHostname = getOrThrow(opts, "hdfs.UseHostName"); @@ -99,7 +107,8 @@ public HdfsConfig(Map opts) { } public HdfsConfig() { - cutoffOffset = 0L; + includeFileEpochAndAfter = 0L; + includeRecordEpochAndAfter = 0L; hdfsPath = ""; hdfsUri = ""; UseHdfsHostname = ""; diff --git a/src/main/java/com/teragrep/pth_06/planner/HdfsDBClient.java b/src/main/java/com/teragrep/pth_06/planner/HdfsDBClient.java index 8358b6f9..79e9f344 100644 --- a/src/main/java/com/teragrep/pth_06/planner/HdfsDBClient.java +++ b/src/main/java/com/teragrep/pth_06/planner/HdfsDBClient.java @@ -72,7 +72,7 @@ public class HdfsDBClient { public HdfsDBClient(Config config, String topicsRegexString) throws IOException { this.topicsRegexString = topicsRegexString; - this.ignoreBeforeEpoch = config.hdfsConfig.cutoffOffset; // Defines the minimum time window / cutoff epoch for making sure that the files which metadata is fetched by the planner are not mistakenly deleted by cfe_39 pruning before tasker can process the records inside. + this.ignoreBeforeEpoch = config.hdfsConfig.includeFileEpochAndAfter; // Defines the minimum time window / cutoff epoch for making sure that the files which metadata is fetched by the planner are not mistakenly deleted by cfe_39 pruning before tasker can process the records inside. // Implement HDFS FileSystem access here if ("kerberos".equals(config.hdfsConfig.kerberosAuthentication)) { // Code for initializing the class with kerberos. diff --git a/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartition.java b/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartition.java index 9f2de4ae..01074dcd 100644 --- a/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartition.java +++ b/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartition.java @@ -59,7 +59,7 @@ public class HdfsMicroBatchInputPartition implements InputPartition { public final LinkedList taskObjectList; - public final long cutoffOffset; // Represents the cutoff epoch which dictates which files should not be fetched from HDFS based on their timestamps. + public final long includeRecordEpochAndAfter; // Represents the cutoff epoch which dictates which files should not be fetched from HDFS based on their timestamps. public final String hdfsPath; // Represents the working directory path in HDFS filesystem. public final String hdfsUri; // Represents the address of the HDFS server. public final String UseHdfsHostname; @@ -83,7 +83,7 @@ public HdfsMicroBatchInputPartition( kerberosAutorenewal = hdfsConfig.kerberosAutorenewal; hdfsUri = hdfsConfig.hdfsUri; hdfsPath = hdfsConfig.hdfsPath; - cutoffOffset = hdfsConfig.cutoffOffset; + includeRecordEpochAndAfter = hdfsConfig.includeRecordEpochAndAfter; UseHdfsHostname = hdfsConfig.UseHdfsHostname; hdfsTransferProtection = hdfsConfig.hdfsTransferProtection; hdfsCipherSuites = hdfsConfig.hdfsCipherSuites; diff --git a/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReader.java b/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReader.java index 2795b0c1..e9a0df41 100644 --- a/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReader.java +++ b/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReader.java @@ -63,9 +63,11 @@ public class HdfsMicroBatchInputPartitionReader implements PartitionReader taskObjectList; private final FileSystem fs; + private final long cutoffEpoch; private HdfsRecordConverter hdfsRecordConverter; public HdfsMicroBatchInputPartitionReader( + long cutoffEpoch, String kerberosAuthentication, String hdfsUri, String hdfsPath, @@ -81,6 +83,7 @@ public HdfsMicroBatchInputPartitionReader( String kerberosKeytabPath, LinkedList taskObjectList ) throws IOException { + this.cutoffEpoch = cutoffEpoch; this.taskObjectList = taskObjectList; // FIXME: Implement FileSystem initialization code somewhere else than the constructor. @@ -170,7 +173,13 @@ public boolean next() throws IOException { hdfsRecordConverter.open(); } } - + else { + // time based inclusion, skip record and continue loop if the record is older than cutoffEpoch. + long rfc5424time = hdfsRecordConverter.get().getLong(0); // timestamp as epochMicros + if (rfc5424time < cutoffEpoch) { + rv = false; + } + } } return rv; } diff --git a/src/main/java/com/teragrep/pth_06/task/TeragrepPartitionReaderFactory.java b/src/main/java/com/teragrep/pth_06/task/TeragrepPartitionReaderFactory.java index e168a0ec..45194082 100644 --- a/src/main/java/com/teragrep/pth_06/task/TeragrepPartitionReaderFactory.java +++ b/src/main/java/com/teragrep/pth_06/task/TeragrepPartitionReaderFactory.java @@ -118,6 +118,7 @@ else if (inputPartition instanceof HdfsMicroBatchInputPartition) { HdfsMicroBatchInputPartition hip = (HdfsMicroBatchInputPartition) inputPartition; try { return new HdfsMicroBatchInputPartitionReader( + hip.includeRecordEpochAndAfter, hip.kerberosAuthentication, hip.hdfsUri, hip.hdfsPath, diff --git a/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReader.java b/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReader.java index 0b0b6094..beb714c8 100644 --- a/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReader.java +++ b/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReader.java @@ -65,6 +65,7 @@ public class AvroReader { private final UnsafeRowWriter rowWriter; private FSDataInputStream inputStream; private DataFileStream reader; + private SyslogRecord currentRecord; // This class will allow reading the contents of the avro-files that are using SyslogRecord schema. public AvroReader(FileSystem fs, HdfsTopicPartitionOffsetMetadata hdfsTopicPartitionOffsetMetadata) { @@ -80,24 +81,26 @@ public void open() throws IOException { } public boolean next() { - return reader.hasNext(); + boolean hasNext = reader.hasNext(); + if (hasNext) { + currentRecord = reader.next(); + } + return hasNext; } public InternalRow get() { - SyslogRecord next = reader.next(); - rowWriter.reset(); rowWriter.zeroOutNullBytes(); - rowWriter.write(0, next.getTimestamp()); - rowWriter.write(1, UTF8String.fromString(next.getPayload().toString())); - rowWriter.write(2, UTF8String.fromString(next.getDirectory().toString())); - rowWriter.write(3, UTF8String.fromString(next.getStream().toString())); - rowWriter.write(4, UTF8String.fromString(next.getHost().toString())); - rowWriter.write(5, UTF8String.fromString(next.getInput().toString())); - rowWriter.write(6, UTF8String.fromString(next.getPartition().toString())); - rowWriter.write(7, UTF8String.fromString(String.valueOf(next.getOffset()))); - rowWriter.write(8, UTF8String.fromString(next.getOrigin().toString())); + rowWriter.write(0, currentRecord.getTimestamp()); + rowWriter.write(1, UTF8String.fromString(currentRecord.getPayload().toString())); + rowWriter.write(2, UTF8String.fromString(currentRecord.getDirectory().toString())); + rowWriter.write(3, UTF8String.fromString(currentRecord.getStream().toString())); + rowWriter.write(4, UTF8String.fromString(currentRecord.getHost().toString())); + rowWriter.write(5, UTF8String.fromString(currentRecord.getInput().toString())); + rowWriter.write(6, UTF8String.fromString(currentRecord.getPartition().toString())); + rowWriter.write(7, UTF8String.fromString(String.valueOf(currentRecord.getOffset()))); + rowWriter.write(8, UTF8String.fromString(currentRecord.getOrigin().toString())); return rowWriter.getRow(); } diff --git a/src/test/java/com/teragrep/pth_06/InstantiationTest.java b/src/test/java/com/teragrep/pth_06/InstantiationTest.java index a5004833..3a16e515 100644 --- a/src/test/java/com/teragrep/pth_06/InstantiationTest.java +++ b/src/test/java/com/teragrep/pth_06/InstantiationTest.java @@ -173,6 +173,7 @@ public void fullScanTest() throws StreamingQueryException, TimeoutException { .option("hdfs.krbKeytabUser", "test") .option("hdfs.krbKeytabPath", "test") .option("hdfs.useKerberosAutorenewal", "true") + .option("hdfs.includeRecordEpochAndAfter", 0L) // record timestamps are in 1650872090806000 granularity (epochMicros) .load(); Dataset df2 = df.agg(functions.count("*")); From b4d58bb8c77a0744bebc4af0e4bb8df3019f8535 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Wed, 18 Sep 2024 15:46:05 +0300 Subject: [PATCH 09/62] Fixed error in InternalRow formatting. --- src/main/java/com/teragrep/pth_06/task/hdfs/AvroReader.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReader.java b/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReader.java index beb714c8..1bf78bc6 100644 --- a/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReader.java +++ b/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReader.java @@ -99,7 +99,7 @@ public InternalRow get() { rowWriter.write(4, UTF8String.fromString(currentRecord.getHost().toString())); rowWriter.write(5, UTF8String.fromString(currentRecord.getInput().toString())); rowWriter.write(6, UTF8String.fromString(currentRecord.getPartition().toString())); - rowWriter.write(7, UTF8String.fromString(String.valueOf(currentRecord.getOffset()))); + rowWriter.write(7, currentRecord.getOffset()); rowWriter.write(8, UTF8String.fromString(currentRecord.getOrigin().toString())); return rowWriter.getRow(); From d8b07d78c15a53819daa812ca272857e39f27006 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Wed, 18 Sep 2024 16:04:40 +0300 Subject: [PATCH 10/62] Implemented HdfsMicroBatchInputPartitionReaderTest.java. Improved MockHDFS functionality for mock data insertion. --- .../teragrep/pth_06/InstantiationTest.java | 3 +- .../com/teragrep/pth_06/planner/MockHDFS.java | 13 +- ...dfsMicroBatchInputPartitionReaderTest.java | 121 ++++++++++++++++++ 3 files changed, 132 insertions(+), 5 deletions(-) create mode 100644 src/test/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReaderTest.java diff --git a/src/test/java/com/teragrep/pth_06/InstantiationTest.java b/src/test/java/com/teragrep/pth_06/InstantiationTest.java index 3a16e515..8af39876 100644 --- a/src/test/java/com/teragrep/pth_06/InstantiationTest.java +++ b/src/test/java/com/teragrep/pth_06/InstantiationTest.java @@ -110,7 +110,7 @@ public void prepareEnv() throws Exception { mockS3.start(); // Start mock hdfs here in a similar way that the mockS3 is implemented. - hdfsUri = mockHDFS.startMiniCluster(); + hdfsUri = mockHDFS.startMiniCluster(false); spark = SparkSession .builder() @@ -299,6 +299,7 @@ private boolean isArchiveDone(StreamingQuery outQ) { @AfterAll public void decommissionEnv() throws Exception { mockS3.stop(); + mockHDFS.teardownMiniCluster(); } private long preloadS3Data() throws IOException { diff --git a/src/test/java/com/teragrep/pth_06/planner/MockHDFS.java b/src/test/java/com/teragrep/pth_06/planner/MockHDFS.java index 840533cf..1e45af52 100644 --- a/src/test/java/com/teragrep/pth_06/planner/MockHDFS.java +++ b/src/test/java/com/teragrep/pth_06/planner/MockHDFS.java @@ -78,7 +78,7 @@ public MockHDFS(String hdfsPath) { } // Start minicluster and initialize config. Returns the hdfsUri of the minicluster. - public String startMiniCluster() throws IOException, InterruptedException { + public String startMiniCluster(boolean insertAll) throws IOException, InterruptedException { // Create a HDFS miniCluster baseDir = Files.createTempDirectory("test_hdfs").toFile().getAbsoluteFile(); @@ -90,7 +90,7 @@ public String startMiniCluster() throws IOException, InterruptedException { // System.out.println("hdfsURI: " + hdfsURI); DistributedFileSystem fileSystem = hdfsCluster.getFileSystem(); - insertMockFiles(); + insertMockFiles(insertAll); return hdfsURI; } @@ -101,7 +101,7 @@ public void teardownMiniCluster() { FileUtil.fullyDelete(baseDir); } - public static void insertMockFiles() throws IOException { + public static void insertMockFiles(boolean insertAll) throws IOException { String path = hdfsPath + "testConsumerTopic"; // "hdfs:///opt/teragrep/cfe_39/srv/testConsumerTopic" // ====== Init HDFS File System Object Configuration conf = new Configuration(); @@ -147,9 +147,14 @@ public static void insertMockFiles() throws IOException { /* This condition can be used for managing the test offsets between HDFS and Kafka. If 0.13 file is included in the HDFS, then Kafka won't have any new records available for reading. If 0.13 is excluded, then records from 0 to 8 will be read from HDFS and records from 9 to 13 will be read from Kafka. Implement a method to decide if all the files should be added to HDFS or not*/ - if (!avroFile.getName().equals("0.13")) { + if (insertAll) { fs.copyFromLocalFile(readPath, hdfswritepath); } + else { + if (!avroFile.getName().equals("0.13")) { + fs.copyFromLocalFile(readPath, hdfswritepath); + } + } LOGGER.debug("End Write file into hdfs"); LOGGER.debug("\nFile committed to HDFS, file writepath should be: {}\n", hdfswritepath.toString()); } diff --git a/src/test/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReaderTest.java b/src/test/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReaderTest.java new file mode 100644 index 00000000..3eedd92b --- /dev/null +++ b/src/test/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReaderTest.java @@ -0,0 +1,121 @@ +/* + * Teragrep Archive Datasource (pth_06) + * 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.pth_06.task; + +import com.teragrep.pth_06.HdfsTopicPartitionOffsetMetadata; +import com.teragrep.pth_06.planner.MockHDFS; +import org.apache.kafka.common.TopicPartition; +import org.apache.spark.sql.catalyst.InternalRow; +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 java.io.IOException; +import java.util.LinkedList; + +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; + +public class HdfsMicroBatchInputPartitionReaderTest { + + private final String hdfsPath = "hdfs:///opt/teragrep/cfe_39/srv/"; + private String hdfsUri; // Can only be defined after starting the mock hdfs. + private final MockHDFS mockHDFS = new MockHDFS(hdfsPath); + + @BeforeEach + public void setUp() throws IOException, InterruptedException { + // Start mock hdfs here in a similar way that the mockS3 is implemented. + hdfsUri = mockHDFS.startMiniCluster(true); + } + + @AfterEach + public void teardown() { + mockHDFS.teardownMiniCluster(); + } + + @Test + public void testHdfsConsumer() { + assertDoesNotThrow(() -> { + // create task object list + LinkedList taskObjectList = new LinkedList<>(); + // Add objects to the taskObjectList according to what files are stored in minicluster during setup. + taskObjectList + .add(new HdfsTopicPartitionOffsetMetadata(new TopicPartition("testConsumerTopic", 0), 9, hdfsUri + "opt/teragrep/cfe_39/srv/testConsumerTopic/0.9", 0)); + taskObjectList + .add(new HdfsTopicPartitionOffsetMetadata(new TopicPartition("testConsumerTopic", 0), 13, hdfsUri + "opt/teragrep/cfe_39/srv/testConsumerTopic/0.13", 0)); + + HdfsMicroBatchInputPartitionReader hdfsMicroBatchInputPartitionReader = new HdfsMicroBatchInputPartitionReader( + 0L, + "", + hdfsUri, + hdfsPath, + "", + "", + "", + "", + "", + "", + "", + "", + "", + "", + taskObjectList + ); + + // read through the files in HDFS + long rowNum = 0L; + while (hdfsMicroBatchInputPartitionReader.next()) { + InternalRow internalRow = hdfsMicroBatchInputPartitionReader.get(); + Assertions.assertEquals(rowNum, internalRow.getLong(7)); // Checks offsets of the consumed records which should range from 0 to 9. + rowNum++; + } + Assertions.assertEquals(14, rowNum); // Asserts that expected number of records were consumed from the files. + + }); + + } + +} From ae3b651df211216b84e68551745c806352cabfdd Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Wed, 18 Sep 2024 16:20:31 +0300 Subject: [PATCH 11/62] Added missing .close() call and fixed error in commenting. --- .../pth_06/task/HdfsMicroBatchInputPartitionReaderTest.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/test/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReaderTest.java b/src/test/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReaderTest.java index 3eedd92b..a0d4d755 100644 --- a/src/test/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReaderTest.java +++ b/src/test/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReaderTest.java @@ -109,10 +109,11 @@ public void testHdfsConsumer() { long rowNum = 0L; while (hdfsMicroBatchInputPartitionReader.next()) { InternalRow internalRow = hdfsMicroBatchInputPartitionReader.get(); - Assertions.assertEquals(rowNum, internalRow.getLong(7)); // Checks offsets of the consumed records which should range from 0 to 9. + Assertions.assertEquals(rowNum, internalRow.getLong(7)); // Checks offsets of the consumed records which should range from 0 to 13. rowNum++; } Assertions.assertEquals(14, rowNum); // Asserts that expected number of records were consumed from the files. + hdfsMicroBatchInputPartitionReader.close(); }); From 03b1ed0606861228976e6ab5abce0676a210158f Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Thu, 19 Sep 2024 09:37:21 +0300 Subject: [PATCH 12/62] Added more tests to HdfsMicroBatchInputPartitionReaderTest.java. --- ...dfsMicroBatchInputPartitionReaderTest.java | 84 ++++++++++++++++++- 1 file changed, 81 insertions(+), 3 deletions(-) diff --git a/src/test/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReaderTest.java b/src/test/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReaderTest.java index a0d4d755..b390ad87 100644 --- a/src/test/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReaderTest.java +++ b/src/test/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReaderTest.java @@ -67,7 +67,6 @@ public class HdfsMicroBatchInputPartitionReaderTest { @BeforeEach public void setUp() throws IOException, InterruptedException { - // Start mock hdfs here in a similar way that the mockS3 is implemented. hdfsUri = mockHDFS.startMiniCluster(true); } @@ -77,11 +76,11 @@ public void teardown() { } @Test - public void testHdfsConsumer() { + public void testHdfsConsumer2Files() { assertDoesNotThrow(() -> { // create task object list LinkedList taskObjectList = new LinkedList<>(); - // Add objects to the taskObjectList according to what files are stored in minicluster during setup. + // Add taskObjects to the taskObjectList according to what files are stored in minicluster during setup. taskObjectList .add(new HdfsTopicPartitionOffsetMetadata(new TopicPartition("testConsumerTopic", 0), 9, hdfsUri + "opt/teragrep/cfe_39/srv/testConsumerTopic/0.9", 0)); taskObjectList @@ -112,9 +111,88 @@ public void testHdfsConsumer() { Assertions.assertEquals(rowNum, internalRow.getLong(7)); // Checks offsets of the consumed records which should range from 0 to 13. rowNum++; } + hdfsMicroBatchInputPartitionReader.close(); Assertions.assertEquals(14, rowNum); // Asserts that expected number of records were consumed from the files. + }); + + } + + @Test + public void testHdfsConsumer1File() { + assertDoesNotThrow(() -> { + // create task object list + LinkedList taskObjectList = new LinkedList<>(); + // Add only the taskObject related to testConsumerTopic/0.9 file to the taskObjectList. + taskObjectList + .add(new HdfsTopicPartitionOffsetMetadata(new TopicPartition("testConsumerTopic", 0), 9, hdfsUri + "opt/teragrep/cfe_39/srv/testConsumerTopic/0.9", 0)); + + HdfsMicroBatchInputPartitionReader hdfsMicroBatchInputPartitionReader = new HdfsMicroBatchInputPartitionReader( + 0L, + "", + hdfsUri, + hdfsPath, + "", + "", + "", + "", + "", + "", + "", + "", + "", + "", + taskObjectList + ); + + // read through the files in HDFS + long rowNum = 0L; + while (hdfsMicroBatchInputPartitionReader.next()) { + InternalRow internalRow = hdfsMicroBatchInputPartitionReader.get(); + Assertions.assertEquals(rowNum, internalRow.getLong(7)); // Checks offsets of the consumed records which should range from 0 to 9. + rowNum++; + } hdfsMicroBatchInputPartitionReader.close(); + Assertions.assertEquals(10, rowNum); // Asserts that expected number of records were consumed from the file. + }); + + } + + @Test + public void testHdfsConsumer1FileAlt() { + assertDoesNotThrow(() -> { + // create task object list + LinkedList taskObjectList = new LinkedList<>(); + // Add only the taskObject related to testConsumerTopic/0.13 file to the taskObjectList. + taskObjectList + .add(new HdfsTopicPartitionOffsetMetadata(new TopicPartition("testConsumerTopic", 0), 13, hdfsUri + "opt/teragrep/cfe_39/srv/testConsumerTopic/0.13", 0)); + HdfsMicroBatchInputPartitionReader hdfsMicroBatchInputPartitionReader = new HdfsMicroBatchInputPartitionReader( + 0L, + "", + hdfsUri, + hdfsPath, + "", + "", + "", + "", + "", + "", + "", + "", + "", + "", + taskObjectList + ); + + // read through the files in HDFS + long rowNum = 10L; + while (hdfsMicroBatchInputPartitionReader.next()) { + InternalRow internalRow = hdfsMicroBatchInputPartitionReader.get(); + Assertions.assertEquals(rowNum, internalRow.getLong(7)); // Checks offsets of the consumed records which should range from 10 to 13. + rowNum++; + } + hdfsMicroBatchInputPartitionReader.close(); + Assertions.assertEquals(14, rowNum); }); } From 4118414be544c5cdf3650c679852d8837a4465ec Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Thu, 19 Sep 2024 12:41:35 +0300 Subject: [PATCH 13/62] Implemented AvroReaderTest.java. Moved hdfs tasker tests to their own directory. Moved the responsibility of converting SyslogRecord to InternalRow from AvroReader to HdfsRecordConverter. --- .../teragrep/pth_06/task/hdfs/AvroReader.java | 24 +--- .../pth_06/task/hdfs/HdfsRecordConverter.java | 23 ++- .../pth_06/task/hdfs/AvroReaderTest.java | 136 ++++++++++++++++++ ...dfsMicroBatchInputPartitionReaderTest.java | 3 +- 4 files changed, 161 insertions(+), 25 deletions(-) create mode 100644 src/test/java/com/teragrep/pth_06/task/hdfs/AvroReaderTest.java rename src/test/java/com/teragrep/pth_06/task/{ => hdfs}/HdfsMicroBatchInputPartitionReaderTest.java (98%) diff --git a/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReader.java b/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReader.java index 1bf78bc6..7b23938d 100644 --- a/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReader.java +++ b/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReader.java @@ -52,17 +52,13 @@ import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter; -import org.apache.spark.unsafe.types.UTF8String; import java.io.IOException; -public class AvroReader { +public final class AvroReader { private final FileSystem fs; private final HdfsTopicPartitionOffsetMetadata hdfsTopicPartitionOffsetMetadata; - private final UnsafeRowWriter rowWriter; private FSDataInputStream inputStream; private DataFileStream reader; private SyslogRecord currentRecord; @@ -71,7 +67,6 @@ public class AvroReader { public AvroReader(FileSystem fs, HdfsTopicPartitionOffsetMetadata hdfsTopicPartitionOffsetMetadata) { this.fs = fs; this.hdfsTopicPartitionOffsetMetadata = hdfsTopicPartitionOffsetMetadata; - this.rowWriter = new UnsafeRowWriter(11); } public void open() throws IOException { @@ -88,21 +83,8 @@ public boolean next() { return hasNext; } - public InternalRow get() { - - rowWriter.reset(); - rowWriter.zeroOutNullBytes(); - rowWriter.write(0, currentRecord.getTimestamp()); - rowWriter.write(1, UTF8String.fromString(currentRecord.getPayload().toString())); - rowWriter.write(2, UTF8String.fromString(currentRecord.getDirectory().toString())); - rowWriter.write(3, UTF8String.fromString(currentRecord.getStream().toString())); - rowWriter.write(4, UTF8String.fromString(currentRecord.getHost().toString())); - rowWriter.write(5, UTF8String.fromString(currentRecord.getInput().toString())); - rowWriter.write(6, UTF8String.fromString(currentRecord.getPartition().toString())); - rowWriter.write(7, currentRecord.getOffset()); - rowWriter.write(8, UTF8String.fromString(currentRecord.getOrigin().toString())); - - return rowWriter.getRow(); + public SyslogRecord get() { + return currentRecord; } public void close() throws IOException { diff --git a/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConverter.java b/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConverter.java index 1772cb63..b085760b 100644 --- a/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConverter.java +++ b/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConverter.java @@ -46,19 +46,23 @@ package com.teragrep.pth_06.task.hdfs; import com.teragrep.pth_06.HdfsTopicPartitionOffsetMetadata; +import com.teragrep.pth_06.avro.SyslogRecord; import org.apache.hadoop.fs.FileSystem; import org.apache.kafka.common.TopicPartition; import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter; +import org.apache.spark.unsafe.types.UTF8String; import java.io.IOException; -// This class will read the records from avro-files fetched from HDFS with the help of AvroReader and convert them to the schema/format used by pth_06. -public class HdfsRecordConverter { +// This class will read the records from avro-files fetched from HDFS with the help of AvroReader and convert them to InternalRow used by pth_06. +public final class HdfsRecordConverter { private final FileSystem fs; private final HdfsTopicPartitionOffsetMetadata hdfsTopicPartitionOffsetMetadata; private final boolean stub; private final AvroReader avroReader; + private final UnsafeRowWriter rowWriter; // Stub object public HdfsRecordConverter(FileSystem fs) { @@ -78,6 +82,7 @@ public HdfsRecordConverter( this.hdfsTopicPartitionOffsetMetadata = hdfsTopicPartitionOffsetMetadata; this.avroReader = new AvroReader(this.fs, this.hdfsTopicPartitionOffsetMetadata); this.stub = stub; + this.rowWriter = new UnsafeRowWriter(11); } public void open() throws IOException { @@ -93,7 +98,19 @@ public boolean next() { } public InternalRow get() { - return avroReader.get(); + SyslogRecord currentRecord = avroReader.get(); + rowWriter.reset(); + rowWriter.zeroOutNullBytes(); + rowWriter.write(0, currentRecord.getTimestamp()); + rowWriter.write(1, UTF8String.fromString(currentRecord.getPayload().toString())); + rowWriter.write(2, UTF8String.fromString(currentRecord.getDirectory().toString())); + rowWriter.write(3, UTF8String.fromString(currentRecord.getStream().toString())); + rowWriter.write(4, UTF8String.fromString(currentRecord.getHost().toString())); + rowWriter.write(5, UTF8String.fromString(currentRecord.getInput().toString())); + rowWriter.write(6, UTF8String.fromString(currentRecord.getPartition().toString())); + rowWriter.write(7, currentRecord.getOffset()); + rowWriter.write(8, UTF8String.fromString(currentRecord.getOrigin().toString())); + return rowWriter.getRow(); } public boolean isStub() { diff --git a/src/test/java/com/teragrep/pth_06/task/hdfs/AvroReaderTest.java b/src/test/java/com/teragrep/pth_06/task/hdfs/AvroReaderTest.java new file mode 100644 index 00000000..b8e1ac83 --- /dev/null +++ b/src/test/java/com/teragrep/pth_06/task/hdfs/AvroReaderTest.java @@ -0,0 +1,136 @@ +/* + * Teragrep Archive Datasource (pth_06) + * 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.pth_06.task.hdfs; + +import com.teragrep.pth_06.HdfsTopicPartitionOffsetMetadata; +import com.teragrep.pth_06.avro.SyslogRecord; +import com.teragrep.pth_06.planner.MockHDFS; +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.kafka.common.TopicPartition; +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 java.io.IOException; +import java.net.URI; + +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; + +public class AvroReaderTest { + + private final String hdfsPath = "hdfs:///opt/teragrep/cfe_39/srv/"; + private String hdfsUri; // Can only be defined after starting the mock hdfs. + private final MockHDFS mockHDFS = new MockHDFS(hdfsPath); + + @BeforeEach + public void setUp() throws IOException, InterruptedException { + hdfsUri = mockHDFS.startMiniCluster(true); + } + + @AfterEach + public void teardown() { + mockHDFS.teardownMiniCluster(); + } + + @Test + public void testReadAllRecords() { + assertDoesNotThrow(() -> { + + // build FileSystem object + // ====== 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 + FileSystem fs = FileSystem.get(URI.create(hdfsUri), conf); + + // build hdfsTopicPartitionOffsetMetadata object + HdfsTopicPartitionOffsetMetadata testConsumerTopic09 = new HdfsTopicPartitionOffsetMetadata( + new TopicPartition("testConsumerTopic", 0), + 9, + hdfsUri + "opt/teragrep/cfe_39/srv/testConsumerTopic/0.9", + 0 + ); + AvroReader avroReader1 = new AvroReader(fs, testConsumerTopic09); + avroReader1.open(); + long rowNum = 0L; + while (avroReader1.next()) { + SyslogRecord syslogRecord = avroReader1.get(); + Assertions.assertEquals(rowNum, syslogRecord.getOffset()); // Checks offsets of the consumed records which should range from 0 to 9. + rowNum++; + } + avroReader1.close(); + Assertions.assertEquals(10, rowNum); // Asserts that expected number of records were consumed from the files. + + HdfsTopicPartitionOffsetMetadata testConsumerTopic013 = new HdfsTopicPartitionOffsetMetadata( + new TopicPartition("testConsumerTopic", 0), + 13, + hdfsUri + "opt/teragrep/cfe_39/srv/testConsumerTopic/0.13", + 0 + ); + AvroReader avroReader2 = new AvroReader(fs, testConsumerTopic013); + avroReader2.open(); + while (avroReader2.next()) { + SyslogRecord syslogRecord = avroReader2.get(); + Assertions.assertEquals(rowNum, syslogRecord.getOffset()); // Checks offsets of the consumed records which should range from 10 to 13. + rowNum++; + } + avroReader2.close(); + Assertions.assertEquals(14, rowNum); // Asserts that expected number of records were consumed from the files. + }); + } + +} diff --git a/src/test/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReaderTest.java b/src/test/java/com/teragrep/pth_06/task/hdfs/HdfsMicroBatchInputPartitionReaderTest.java similarity index 98% rename from src/test/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReaderTest.java rename to src/test/java/com/teragrep/pth_06/task/hdfs/HdfsMicroBatchInputPartitionReaderTest.java index b390ad87..57e687f9 100644 --- a/src/test/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReaderTest.java +++ b/src/test/java/com/teragrep/pth_06/task/hdfs/HdfsMicroBatchInputPartitionReaderTest.java @@ -43,10 +43,11 @@ * Teragrep, the applicable Commercial License may apply to this file if you as * a licensee so wish it. */ -package com.teragrep.pth_06.task; +package com.teragrep.pth_06.task.hdfs; import com.teragrep.pth_06.HdfsTopicPartitionOffsetMetadata; import com.teragrep.pth_06.planner.MockHDFS; +import com.teragrep.pth_06.task.HdfsMicroBatchInputPartitionReader; import org.apache.kafka.common.TopicPartition; import org.apache.spark.sql.catalyst.InternalRow; import org.junit.jupiter.api.AfterEach; From 3b3c70387564cf491012700b7bfc260e77a2e245 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Thu, 19 Sep 2024 14:41:48 +0300 Subject: [PATCH 14/62] Removed obsolete test code from MockHDFS. --- .../com/teragrep/pth_06/planner/MockHDFS.java | 131 ------------------ 1 file changed, 131 deletions(-) diff --git a/src/test/java/com/teragrep/pth_06/planner/MockHDFS.java b/src/test/java/com/teragrep/pth_06/planner/MockHDFS.java index 1e45af52..fff53799 100644 --- a/src/test/java/com/teragrep/pth_06/planner/MockHDFS.java +++ b/src/test/java/com/teragrep/pth_06/planner/MockHDFS.java @@ -45,15 +45,10 @@ */ package com.teragrep.pth_06.planner; -import com.google.gson.JsonArray; -import com.google.gson.JsonElement; -import com.google.gson.JsonObject; 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.apache.kafka.common.TopicPartition; -import org.junit.jupiter.api.Assertions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -160,130 +155,4 @@ public static void insertMockFiles(boolean insertAll) throws IOException { } fs.close(); } - - public void simpleTest() { - try { - hdfsReadTest(); - } - catch (IOException e) { - throw new RuntimeException(e); - } - } - - public void hdfsReadTest() throws IOException { - Map hdfsStartOffsets = new HashMap<>(); - // Check that the files were properly written to HDFS with a read test. - String path = "hdfs:///opt/teragrep/cfe_39/srv/"; // This path parameter should only lead to the directory that holds the directories representing different Kafka topics. - // ====== 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); - } - FileStatus[] directoryStatuses = fs.listStatus(newDirectoryPath, pathFilter); - if (directoryStatuses.length > 0) { - for (FileStatus r : directoryStatuses) { - LOGGER.info("The path to the directory is: {}", r.getPath()); - LOGGER.info("The directory name is: {}", r.getPath().getName()); - 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]; - LOGGER.info("File belongs to topic: {}", topic); - LOGGER.info("File belongs to partition: {}", partition); - LOGGER.info("File has an offset of: {}", offset); - if (!hdfsStartOffsets.containsKey(new TopicPartition(topic, Integer.parseInt(partition)))) { - hdfsStartOffsets - .put(new TopicPartition(topic, Integer.parseInt(partition)), Long.parseLong(offset)); - } - else { - if ( - hdfsStartOffsets.get(new TopicPartition(topic, Integer.parseInt(partition))) < Long - .parseLong(offset) - ) { - hdfsStartOffsets - .replace(new TopicPartition(topic, Integer.parseInt(partition)), Long.parseLong(offset)); - } - } - } - } - } - else { - LOGGER.info("No matching directories found"); - } - LOGGER.info("hdfsStartOffsets.toString(): "); - LOGGER.info(hdfsStartOffsets.toString()); - - LinkedList rv = new LinkedList<>(); - JsonArray ja = new JsonArray(); - // Serializing the hdfsStartOffsets to a JsonArray ja. - for (Map.Entry entry : hdfsStartOffsets.entrySet()) { - String topic = entry.getKey().topic(); - String partition = String.valueOf(entry.getKey().partition()); - String offset = String.valueOf(entry.getValue()); - rv - .add( - String - .format( - "{\"topic\":\"%s\", \"partition\":\"%s\", \"offset\":\"%s\"}", topic, - partition, offset - ) - ); - JsonObject jo = new JsonObject(); // TODO: Use this instead of string - jo.addProperty("topic", topic); - jo.addProperty("partition", partition); - jo.addProperty("offset", offset); - ja.add(jo); - } - // LOGGER.info(rv.toString()); - LOGGER.info("ja.toString(): "); - LOGGER.info(ja.toString()); - - // Deserialize ja back to Map - Map offsetMap = new HashMap<>(); - for (JsonElement pa : ja) { - JsonObject offsetObject = pa.getAsJsonObject(); - TopicPartition topicPartition = new TopicPartition( - offsetObject.get("topic").getAsString(), - offsetObject.get("partition").getAsInt() - ); - Long offset = offsetObject.get("offset").getAsLong(); - offsetMap.put(topicPartition, offset); - } - - LOGGER.info("offsetMap.toString(): "); - LOGGER.info(offsetMap.toString()); - // Assert that hdfsStartOffsets and offsetMap are identical after the json serialization and deserialization cycle of hdfsStartOffsets. - Assertions.assertEquals(hdfsStartOffsets, offsetMap); - - // ja.toString() now outputs: - // [{"topic":"testConsumerTopic","partition":"7","offset":"13"},{"topic":"testConsumerTopic","partition":"8","offset":"13"},{"topic":"testConsumerTopic","partition":"5","offset":"13"},{"topic":"testConsumerTopic","partition":"6","offset":"13"},{"topic":"testConsumerTopic","partition":"3","offset":"13"},{"topic":"testConsumerTopic","partition":"4","offset":"13"},{"topic":"testConsumerTopic","partition":"1","offset":"13"},{"topic":"testConsumerTopic","partition":"2","offset":"13"},{"topic":"testConsumerTopic","partition":"0","offset":"13"},{"topic":"testConsumerTopic","partition":"9","offset":"13"}] - fs.close(); - } - - private static final PathFilter pathFilter = new PathFilter() { - - @Override - public boolean accept(Path path) { - return path.getName().matches("^testConsumer.*$"); // Catches the directory names. - } - }; - } From baf2ddee82d04e237f94f79d73121c38c8990570 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Thu, 19 Sep 2024 14:44:58 +0300 Subject: [PATCH 15/62] Refactored HdfsRecordConverter class to HdfsRecordConversionImpl class which implements HdfsRecordConversion interface. --- .../HdfsMicroBatchInputPartitionReader.java | 28 ++++---- .../task/hdfs/HdfsRecordConversion.java | 64 +++++++++++++++++++ ...ter.java => HdfsRecordConversionImpl.java} | 13 ++-- 3 files changed, 87 insertions(+), 18 deletions(-) create mode 100644 src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversion.java rename src/main/java/com/teragrep/pth_06/task/hdfs/{HdfsRecordConverter.java => HdfsRecordConversionImpl.java} (92%) diff --git a/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReader.java b/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReader.java index e9a0df41..d3705ef7 100644 --- a/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReader.java +++ b/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReader.java @@ -46,7 +46,7 @@ package com.teragrep.pth_06.task; import com.teragrep.pth_06.HdfsTopicPartitionOffsetMetadata; -import com.teragrep.pth_06.task.hdfs.HdfsRecordConverter; +import com.teragrep.pth_06.task.hdfs.HdfsRecordConversionImpl; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.LocalFileSystem; @@ -64,7 +64,7 @@ public class HdfsMicroBatchInputPartitionReader implements PartitionReader taskObjectList; private final FileSystem fs; private final long cutoffEpoch; - private HdfsRecordConverter hdfsRecordConverter; + private HdfsRecordConversionImpl hdfsRecordConversionImpl; public HdfsMicroBatchInputPartitionReader( long cutoffEpoch, @@ -145,7 +145,7 @@ These values should be fetched from config and other input parameters (topic+par // filesystem for HDFS access is set here fs = FileSystem.get(URI.create(hdfsUri), conf); } - this.hdfsRecordConverter = new HdfsRecordConverter(fs); // stub + this.hdfsRecordConversionImpl = new HdfsRecordConversionImpl(fs); // stub } // Read avro-file until it ends @@ -155,27 +155,27 @@ public boolean next() throws IOException { boolean rv = false; while (!taskObjectList.isEmpty() && !rv) { - if (hdfsRecordConverter.isStub()) { - hdfsRecordConverter = new HdfsRecordConverter(fs, taskObjectList.getFirst()); - hdfsRecordConverter.open(); + if (hdfsRecordConversionImpl.isStub()) { + hdfsRecordConversionImpl = new HdfsRecordConversionImpl(fs, taskObjectList.getFirst()); + hdfsRecordConversionImpl.open(); } - rv = hdfsRecordConverter.next(); + rv = hdfsRecordConversionImpl.next(); if (!rv) { // object was consumed - hdfsRecordConverter.close(); + hdfsRecordConversionImpl.close(); // remove consumed object taskObjectList.removeFirst(); if (!taskObjectList.isEmpty()) { - hdfsRecordConverter = new HdfsRecordConverter(fs, taskObjectList.getFirst()); - hdfsRecordConverter.open(); + hdfsRecordConversionImpl = new HdfsRecordConversionImpl(fs, taskObjectList.getFirst()); + hdfsRecordConversionImpl.open(); } } else { // time based inclusion, skip record and continue loop if the record is older than cutoffEpoch. - long rfc5424time = hdfsRecordConverter.get().getLong(0); // timestamp as epochMicros + long rfc5424time = hdfsRecordConversionImpl.get().getLong(0); // timestamp as epochMicros if (rfc5424time < cutoffEpoch) { rv = false; } @@ -186,13 +186,13 @@ public boolean next() throws IOException { @Override public InternalRow get() { - return hdfsRecordConverter.get(); + return hdfsRecordConversionImpl.get(); } @Override public void close() throws IOException { - if (!hdfsRecordConverter.isStub()) { - hdfsRecordConverter.close(); + if (!hdfsRecordConversionImpl.isStub()) { + hdfsRecordConversionImpl.close(); } } } diff --git a/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversion.java b/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversion.java new file mode 100644 index 00000000..0e1fb400 --- /dev/null +++ b/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversion.java @@ -0,0 +1,64 @@ +/* + * Teragrep Archive Datasource (pth_06) + * 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.pth_06.task.hdfs; + +import org.apache.spark.sql.catalyst.InternalRow; + +import java.io.IOException; + +public interface HdfsRecordConversion { + + void open() throws IOException; + + void close() throws IOException; + + boolean next(); + + InternalRow get(); + + boolean isStub(); + +} diff --git a/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConverter.java b/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversionImpl.java similarity index 92% rename from src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConverter.java rename to src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversionImpl.java index b085760b..63c40075 100644 --- a/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConverter.java +++ b/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversionImpl.java @@ -56,7 +56,7 @@ import java.io.IOException; // This class will read the records from avro-files fetched from HDFS with the help of AvroReader and convert them to InternalRow used by pth_06. -public final class HdfsRecordConverter { +public final class HdfsRecordConversionImpl implements HdfsRecordConversion { private final FileSystem fs; private final HdfsTopicPartitionOffsetMetadata hdfsTopicPartitionOffsetMetadata; @@ -65,15 +65,15 @@ public final class HdfsRecordConverter { private final UnsafeRowWriter rowWriter; // Stub object - public HdfsRecordConverter(FileSystem fs) { + public HdfsRecordConversionImpl(FileSystem fs) { this(fs, new HdfsTopicPartitionOffsetMetadata(new TopicPartition("", 0), 0, "", 0), true); } - public HdfsRecordConverter(FileSystem fs, HdfsTopicPartitionOffsetMetadata hdfsTopicPartitionOffsetMetadata) { + public HdfsRecordConversionImpl(FileSystem fs, HdfsTopicPartitionOffsetMetadata hdfsTopicPartitionOffsetMetadata) { this(fs, hdfsTopicPartitionOffsetMetadata, false); } - public HdfsRecordConverter( + public HdfsRecordConversionImpl( FileSystem fs, HdfsTopicPartitionOffsetMetadata hdfsTopicPartitionOffsetMetadata, Boolean stub @@ -85,18 +85,22 @@ public HdfsRecordConverter( this.rowWriter = new UnsafeRowWriter(11); } + @Override public void open() throws IOException { avroReader.open(); } + @Override public void close() throws IOException { avroReader.close(); } + @Override public boolean next() { return avroReader.next(); } + @Override public InternalRow get() { SyslogRecord currentRecord = avroReader.get(); rowWriter.reset(); @@ -113,6 +117,7 @@ public InternalRow get() { return rowWriter.getRow(); } + @Override public boolean isStub() { return stub; } From 388aabac733dc784f14d4b014c20c70892e6aae8 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Thu, 19 Sep 2024 15:37:40 +0300 Subject: [PATCH 16/62] Implemented FileSystemFactoryImpl and FileSystemFactory interface. --- .../teragrep/pth_06/FileSystemFactory.java | 56 ++++++ .../pth_06/FileSystemFactoryImpl.java | 174 ++++++++++++++++++ 2 files changed, 230 insertions(+) create mode 100644 src/main/java/com/teragrep/pth_06/FileSystemFactory.java create mode 100644 src/main/java/com/teragrep/pth_06/FileSystemFactoryImpl.java diff --git a/src/main/java/com/teragrep/pth_06/FileSystemFactory.java b/src/main/java/com/teragrep/pth_06/FileSystemFactory.java new file mode 100644 index 00000000..398736af --- /dev/null +++ b/src/main/java/com/teragrep/pth_06/FileSystemFactory.java @@ -0,0 +1,56 @@ +/* + * Teragrep Archive Datasource (pth_06) + * 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.pth_06; + +import org.apache.hadoop.fs.FileSystem; + +import java.io.IOException; + +public interface FileSystemFactory { + + FileSystem create(boolean initializeUGI) throws IOException; + +} diff --git a/src/main/java/com/teragrep/pth_06/FileSystemFactoryImpl.java b/src/main/java/com/teragrep/pth_06/FileSystemFactoryImpl.java new file mode 100644 index 00000000..c31493cb --- /dev/null +++ b/src/main/java/com/teragrep/pth_06/FileSystemFactoryImpl.java @@ -0,0 +1,174 @@ +/* + * Teragrep Archive Datasource (pth_06) + * 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.pth_06; + +import com.teragrep.pth_06.config.HdfsConfig; +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 java.io.IOException; +import java.net.URI; + +public final class FileSystemFactoryImpl implements FileSystemFactory { + + private final String kerberosAuthentication; + private final String hdfsUri; + private final String kerberosRealm; + private final String kerberosKdc; + private final String kerberosAuthorization; + private final String kerberosAutorenewal; + private final String UseHdfsHostname; + private final String kerberosPrincipalPattern; + private final String hdfsTransferProtection; + private final String hdfsCipherSuites; + private final String kerberosKeytabUser; + private final String kerberosKeytabPath; + + public FileSystemFactoryImpl(HdfsConfig config) { + this( + config.kerberosAuthentication, + config.hdfsUri, + config.kerberosRealm, + config.kerberosKdc, + config.kerberosAuthorization, + config.kerberosAutorenewal, + config.UseHdfsHostname, + config.kerberosPrincipalPattern, + config.hdfsTransferProtection, + config.hdfsCipherSuites, + config.kerberosKeytabUser, + config.kerberosKeytabPath + ); + } + + public FileSystemFactoryImpl( + String kerberosAuthentication, + String hdfsUri, + String kerberosRealm, + String kerberosKdc, + String kerberosAuthorization, + String kerberosAutorenewal, + String UseHdfsHostname, + String kerberosPrincipalPattern, + String hdfsTransferProtection, + String hdfsCipherSuites, + String kerberosKeytabUser, + String kerberosKeytabPath + ) { + this.kerberosAuthentication = kerberosAuthentication; + this.hdfsUri = hdfsUri; + this.kerberosRealm = kerberosRealm; + this.kerberosKdc = kerberosKdc; + this.kerberosAuthorization = kerberosAuthorization; + this.kerberosAutorenewal = kerberosAutorenewal; + this.UseHdfsHostname = UseHdfsHostname; + this.kerberosPrincipalPattern = kerberosPrincipalPattern; + this.hdfsTransferProtection = hdfsTransferProtection; + this.hdfsCipherSuites = hdfsCipherSuites; + this.kerberosKeytabUser = kerberosKeytabUser; + this.kerberosKeytabPath = kerberosKeytabPath; + } + + @Override + public FileSystem create(boolean initializeUGI) throws IOException { + FileSystem fs; + if ("kerberos".equals(kerberosAuthentication)) { + // Code for initializing the FileSystem with kerberos. + + // set kerberos host and realm + System.setProperty("java.security.krb5.realm", kerberosRealm); + System.setProperty("java.security.krb5.kdc", kerberosKdc); + + Configuration conf = new Configuration(); + + // enable kerberus + conf.set("hadoop.security.authentication", kerberosAuthentication); + conf.set("hadoop.security.authorization", kerberosAuthorization); + conf.set("hadoop.kerberos.keytab.login.autorenewal.enabled", kerberosAutorenewal); + + 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", UseHdfsHostname); + + // server principal, the kerberos principle that the namenode is using + conf.set("dfs.namenode.kerberos.principal.pattern", kerberosPrincipalPattern); + + conf.set("dfs.data.transfer.protection", hdfsTransferProtection); + conf.set("dfs.encrypt.data.transfer.cipher.suites", hdfsCipherSuites); + + if (initializeUGI) { + UserGroupInformation.setConfiguration(conf); + UserGroupInformation.loginUserFromKeytab(kerberosKeytabUser, kerberosKeytabPath); + } + + // filesystem for HDFS access is set here + fs = FileSystem.get(conf); + } + else { + // Code for initializing the FileSystem without kerberos. + + // ====== 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 + fs = FileSystem.get(URI.create(hdfsUri), conf); + } + return fs; + } +} From f8047d0928c4811438a5c673afda97f55eb6ce85 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Thu, 19 Sep 2024 15:51:38 +0300 Subject: [PATCH 17/62] Moved FileSystem initialization logic from constructors of HdfsDBClient and HdfsMicroBatchInputPartitionReader to FileSystemFactoryImpl. --- .../teragrep/pth_06/planner/HdfsDBClient.java | 74 +---------------- .../HdfsMicroBatchInputPartitionReader.java | 79 ++++--------------- 2 files changed, 19 insertions(+), 134 deletions(-) diff --git a/src/main/java/com/teragrep/pth_06/planner/HdfsDBClient.java b/src/main/java/com/teragrep/pth_06/planner/HdfsDBClient.java index 79e9f344..d265022d 100644 --- a/src/main/java/com/teragrep/pth_06/planner/HdfsDBClient.java +++ b/src/main/java/com/teragrep/pth_06/planner/HdfsDBClient.java @@ -45,18 +45,15 @@ */ package com.teragrep.pth_06.planner; +import com.teragrep.pth_06.FileSystemFactoryImpl; import com.teragrep.pth_06.config.Config; import com.teragrep.pth_06.HdfsTopicPartitionOffsetMetadata; -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.LinkedList; // Searches the HDFS database for files contained in directories based on topicsRegexString. Creates a list of objects where each object contains the metadata of a single HDFS-file. @@ -65,79 +62,14 @@ public class HdfsDBClient { private final Logger LOGGER = LoggerFactory.getLogger(HdfsDBClient.class); private final String path; private final FileSystem fs; - private final Configuration conf; - private final String hdfsuri; private final String topicsRegexString; private final long ignoreBeforeEpoch; public HdfsDBClient(Config config, String topicsRegexString) throws IOException { this.topicsRegexString = topicsRegexString; this.ignoreBeforeEpoch = config.hdfsConfig.includeFileEpochAndAfter; // Defines the minimum time window / cutoff epoch for making sure that the files which metadata is fetched by the planner are not mistakenly deleted by cfe_39 pruning before tasker can process the records inside. - // Implement HDFS FileSystem access here - if ("kerberos".equals(config.hdfsConfig.kerberosAuthentication)) { - // Code for initializing the class with kerberos. - hdfsuri = config.hdfsConfig.hdfsUri; // 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.hdfsConfig.hdfsPath; - - // set kerberos host and realm - System.setProperty("java.security.krb5.realm", config.hdfsConfig.kerberosRealm); - System.setProperty("java.security.krb5.kdc", config.hdfsConfig.kerberosKdc); - - conf = new Configuration(); - - // enable kerberus - conf.set("hadoop.security.authentication", config.hdfsConfig.kerberosAuthentication); - conf.set("hadoop.security.authorization", config.hdfsConfig.kerberosAuthorization); - conf.set("hadoop.kerberos.keytab.login.autorenewal.enabled", config.hdfsConfig.kerberosAutorenewal); - - 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.hdfsConfig.UseHdfsHostname); - - // server principal, the kerberos principle that the namenode is using - conf.set("dfs.namenode.kerberos.principal.pattern", config.hdfsConfig.kerberosPrincipalPattern); - - conf.set("dfs.data.transfer.protection", config.hdfsConfig.hdfsTransferProtection); - conf.set("dfs.encrypt.data.transfer.cipher.suites", config.hdfsConfig.hdfsCipherSuites); - - // set usergroup stuff - UserGroupInformation.setConfiguration(conf); - UserGroupInformation - .loginUserFromKeytab(config.hdfsConfig.kerberosKeytabUser, config.hdfsConfig.kerberosKeytabPath); - - // filesystem for HDFS access is set here - fs = FileSystem.get(conf); - } - else { - // Code for initializing the class in test mode without kerberos. - hdfsuri = config.hdfsConfig.hdfsUri; // 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. - These values should be fetched from config and other input parameters (topic+partition+offset).*/ - path = config.hdfsConfig.hdfsPath; // Example path: hdfs://localhost:34509/opt/teragrep/cfe_39/srv/ - - // ====== 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 - fs = FileSystem.get(URI.create(hdfsuri), conf); - - } + path = config.hdfsConfig.hdfsPath; + fs = new FileSystemFactoryImpl(config.hdfsConfig).create(true); } // this queries and pulls the distinct file metadata values to the partitionList according to the given query conditions (condition only applies to topic names in planner side). diff --git a/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReader.java b/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReader.java index d3705ef7..76e46e04 100644 --- a/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReader.java +++ b/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReader.java @@ -45,18 +45,14 @@ */ package com.teragrep.pth_06.task; +import com.teragrep.pth_06.FileSystemFactoryImpl; import com.teragrep.pth_06.HdfsTopicPartitionOffsetMetadata; import com.teragrep.pth_06.task.hdfs.HdfsRecordConversionImpl; -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.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.connector.read.PartitionReader; import java.io.IOException; -import java.net.URI; import java.util.LinkedList; public class HdfsMicroBatchInputPartitionReader implements PartitionReader { @@ -83,68 +79,25 @@ public HdfsMicroBatchInputPartitionReader( String kerberosKeytabPath, LinkedList taskObjectList ) throws IOException { + this.cutoffEpoch = cutoffEpoch; this.taskObjectList = taskObjectList; - // FIXME: Implement FileSystem initialization code somewhere else than the constructor. - if ("kerberos".equals(kerberosAuthentication)) { - // Code for initializing the class with 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. - 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).*/ - - // set kerberos host and realm - System.setProperty("java.security.krb5.realm", kerberosRealm); - System.setProperty("java.security.krb5.kdc", kerberosKdc); - - Configuration conf = new Configuration(); - - // enable kerberus - conf.set("hadoop.security.authentication", kerberosAuthentication); - conf.set("hadoop.security.authorization", kerberosAuthorization); - conf.set("hadoop.kerberos.keytab.login.autorenewal.enabled", kerberosAutorenewal); - - 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", UseHdfsHostname); + fs = new FileSystemFactoryImpl( + kerberosAuthentication, + hdfsUri, + kerberosRealm, + kerberosKdc, + kerberosAuthorization, + kerberosAutorenewal, + UseHdfsHostname, + kerberosPrincipalPattern, + hdfsTransferProtection, + hdfsCipherSuites, + kerberosKeytabUser, + kerberosKeytabPath + ).create(false); - // server principal, the kerberos principle that the namenode is using - conf.set("dfs.namenode.kerberos.principal.pattern", kerberosPrincipalPattern); - - conf.set("dfs.data.transfer.protection", hdfsTransferProtection); - conf.set("dfs.encrypt.data.transfer.cipher.suites", hdfsCipherSuites); - - // set usergroup stuff - UserGroupInformation.setConfiguration(conf); - UserGroupInformation.loginUserFromKeytab(kerberosKeytabUser, kerberosKeytabPath); - - // filesystem for HDFS access is set here - fs = FileSystem.get(conf); - } - else { - // Code for initializing the class in test mode without 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. - 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).*/ - - // ====== 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 - fs = FileSystem.get(URI.create(hdfsUri), conf); - } this.hdfsRecordConversionImpl = new HdfsRecordConversionImpl(fs); // stub } From 15a9849749adc67b5e78e3f26125e0435148922a Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Thu, 19 Sep 2024 16:39:51 +0300 Subject: [PATCH 18/62] Refactored class attributes into local variables. --- src/main/java/com/teragrep/pth_06/task/hdfs/AvroReader.java | 5 ++--- .../teragrep/pth_06/task/hdfs/HdfsRecordConversionImpl.java | 6 +----- 2 files changed, 3 insertions(+), 8 deletions(-) diff --git a/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReader.java b/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReader.java index 7b23938d..882c202a 100644 --- a/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReader.java +++ b/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReader.java @@ -55,15 +55,14 @@ import java.io.IOException; +// This class will allow reading the contents of the avro-files that are using SyslogRecord schema from hdfs. public final class AvroReader { private final FileSystem fs; private final HdfsTopicPartitionOffsetMetadata hdfsTopicPartitionOffsetMetadata; - private FSDataInputStream inputStream; private DataFileStream reader; private SyslogRecord currentRecord; - // This class will allow reading the contents of the avro-files that are using SyslogRecord schema. public AvroReader(FileSystem fs, HdfsTopicPartitionOffsetMetadata hdfsTopicPartitionOffsetMetadata) { this.fs = fs; this.hdfsTopicPartitionOffsetMetadata = hdfsTopicPartitionOffsetMetadata; @@ -71,7 +70,7 @@ public AvroReader(FileSystem fs, HdfsTopicPartitionOffsetMetadata hdfsTopicParti public void open() throws IOException { Path hdfsreadpath = new Path(hdfsTopicPartitionOffsetMetadata.hdfsFilePath); - inputStream = fs.open(hdfsreadpath); + FSDataInputStream inputStream = fs.open(hdfsreadpath); reader = new DataFileStream<>(inputStream, new SpecificDatumReader<>(SyslogRecord.class)); } diff --git a/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversionImpl.java b/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversionImpl.java index 63c40075..e4856597 100644 --- a/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversionImpl.java +++ b/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversionImpl.java @@ -58,8 +58,6 @@ // This class will read the records from avro-files fetched from HDFS with the help of AvroReader and convert them to InternalRow used by pth_06. public final class HdfsRecordConversionImpl implements HdfsRecordConversion { - private final FileSystem fs; - private final HdfsTopicPartitionOffsetMetadata hdfsTopicPartitionOffsetMetadata; private final boolean stub; private final AvroReader avroReader; private final UnsafeRowWriter rowWriter; @@ -78,9 +76,7 @@ public HdfsRecordConversionImpl( HdfsTopicPartitionOffsetMetadata hdfsTopicPartitionOffsetMetadata, Boolean stub ) { - this.fs = fs; - this.hdfsTopicPartitionOffsetMetadata = hdfsTopicPartitionOffsetMetadata; - this.avroReader = new AvroReader(this.fs, this.hdfsTopicPartitionOffsetMetadata); + this.avroReader = new AvroReader(fs, hdfsTopicPartitionOffsetMetadata); this.stub = stub; this.rowWriter = new UnsafeRowWriter(11); } From 3aa980fcdd0985e30e26a0d3a980669b60ee56ad Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Fri, 20 Sep 2024 10:04:57 +0300 Subject: [PATCH 19/62] Renamed AvroReader to AvroReadImpl. Implemented AvroRead interface. --- .../teragrep/pth_06/task/hdfs/AvroRead.java | 62 +++++++++++++++++++ .../{AvroReader.java => AvroReadImpl.java} | 8 ++- .../task/hdfs/HdfsRecordConversionImpl.java | 14 ++--- ...oReaderTest.java => AvroReadImplTest.java} | 22 +++---- 4 files changed, 86 insertions(+), 20 deletions(-) create mode 100644 src/main/java/com/teragrep/pth_06/task/hdfs/AvroRead.java rename src/main/java/com/teragrep/pth_06/task/hdfs/{AvroReader.java => AvroReadImpl.java} (93%) rename src/test/java/com/teragrep/pth_06/task/hdfs/{AvroReaderTest.java => AvroReadImplTest.java} (90%) diff --git a/src/main/java/com/teragrep/pth_06/task/hdfs/AvroRead.java b/src/main/java/com/teragrep/pth_06/task/hdfs/AvroRead.java new file mode 100644 index 00000000..67d813f4 --- /dev/null +++ b/src/main/java/com/teragrep/pth_06/task/hdfs/AvroRead.java @@ -0,0 +1,62 @@ +/* + * Teragrep Archive Datasource (pth_06) + * 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.pth_06.task.hdfs; + +import com.teragrep.pth_06.avro.SyslogRecord; + +import java.io.IOException; + +public interface AvroRead { + + void open() throws IOException; + + boolean next(); + + SyslogRecord get(); + + void close() throws IOException; + +} diff --git a/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReader.java b/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReadImpl.java similarity index 93% rename from src/main/java/com/teragrep/pth_06/task/hdfs/AvroReader.java rename to src/main/java/com/teragrep/pth_06/task/hdfs/AvroReadImpl.java index 882c202a..1312f441 100644 --- a/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReader.java +++ b/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReadImpl.java @@ -56,24 +56,26 @@ import java.io.IOException; // This class will allow reading the contents of the avro-files that are using SyslogRecord schema from hdfs. -public final class AvroReader { +public final class AvroReadImpl implements AvroRead { private final FileSystem fs; private final HdfsTopicPartitionOffsetMetadata hdfsTopicPartitionOffsetMetadata; private DataFileStream reader; private SyslogRecord currentRecord; - public AvroReader(FileSystem fs, HdfsTopicPartitionOffsetMetadata hdfsTopicPartitionOffsetMetadata) { + public AvroReadImpl(FileSystem fs, HdfsTopicPartitionOffsetMetadata hdfsTopicPartitionOffsetMetadata) { this.fs = fs; this.hdfsTopicPartitionOffsetMetadata = hdfsTopicPartitionOffsetMetadata; } + @Override public void open() throws IOException { Path hdfsreadpath = new Path(hdfsTopicPartitionOffsetMetadata.hdfsFilePath); FSDataInputStream inputStream = fs.open(hdfsreadpath); reader = new DataFileStream<>(inputStream, new SpecificDatumReader<>(SyslogRecord.class)); } + @Override public boolean next() { boolean hasNext = reader.hasNext(); if (hasNext) { @@ -82,10 +84,12 @@ public boolean next() { return hasNext; } + @Override public SyslogRecord get() { return currentRecord; } + @Override public void close() throws IOException { reader.close(); } diff --git a/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversionImpl.java b/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversionImpl.java index e4856597..0e8a396d 100644 --- a/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversionImpl.java +++ b/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversionImpl.java @@ -55,11 +55,11 @@ import java.io.IOException; -// This class will read the records from avro-files fetched from HDFS with the help of AvroReader and convert them to InternalRow used by pth_06. +// This class will read the records from avro-files fetched from HDFS with the help of AvroReadImpl and convert them to InternalRow used by pth_06. public final class HdfsRecordConversionImpl implements HdfsRecordConversion { private final boolean stub; - private final AvroReader avroReader; + private final AvroReadImpl avroReadImpl; private final UnsafeRowWriter rowWriter; // Stub object @@ -76,29 +76,29 @@ public HdfsRecordConversionImpl( HdfsTopicPartitionOffsetMetadata hdfsTopicPartitionOffsetMetadata, Boolean stub ) { - this.avroReader = new AvroReader(fs, hdfsTopicPartitionOffsetMetadata); + this.avroReadImpl = new AvroReadImpl(fs, hdfsTopicPartitionOffsetMetadata); this.stub = stub; this.rowWriter = new UnsafeRowWriter(11); } @Override public void open() throws IOException { - avroReader.open(); + avroReadImpl.open(); } @Override public void close() throws IOException { - avroReader.close(); + avroReadImpl.close(); } @Override public boolean next() { - return avroReader.next(); + return avroReadImpl.next(); } @Override public InternalRow get() { - SyslogRecord currentRecord = avroReader.get(); + SyslogRecord currentRecord = avroReadImpl.get(); rowWriter.reset(); rowWriter.zeroOutNullBytes(); rowWriter.write(0, currentRecord.getTimestamp()); diff --git a/src/test/java/com/teragrep/pth_06/task/hdfs/AvroReaderTest.java b/src/test/java/com/teragrep/pth_06/task/hdfs/AvroReadImplTest.java similarity index 90% rename from src/test/java/com/teragrep/pth_06/task/hdfs/AvroReaderTest.java rename to src/test/java/com/teragrep/pth_06/task/hdfs/AvroReadImplTest.java index b8e1ac83..cdb56785 100644 --- a/src/test/java/com/teragrep/pth_06/task/hdfs/AvroReaderTest.java +++ b/src/test/java/com/teragrep/pth_06/task/hdfs/AvroReadImplTest.java @@ -63,7 +63,7 @@ import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; -public class AvroReaderTest { +public class AvroReadImplTest { private final String hdfsPath = "hdfs:///opt/teragrep/cfe_39/srv/"; private String hdfsUri; // Can only be defined after starting the mock hdfs. @@ -104,15 +104,15 @@ public void testReadAllRecords() { hdfsUri + "opt/teragrep/cfe_39/srv/testConsumerTopic/0.9", 0 ); - AvroReader avroReader1 = new AvroReader(fs, testConsumerTopic09); - avroReader1.open(); + AvroReadImpl avroReadImpl1 = new AvroReadImpl(fs, testConsumerTopic09); + avroReadImpl1.open(); long rowNum = 0L; - while (avroReader1.next()) { - SyslogRecord syslogRecord = avroReader1.get(); + while (avroReadImpl1.next()) { + SyslogRecord syslogRecord = avroReadImpl1.get(); Assertions.assertEquals(rowNum, syslogRecord.getOffset()); // Checks offsets of the consumed records which should range from 0 to 9. rowNum++; } - avroReader1.close(); + avroReadImpl1.close(); Assertions.assertEquals(10, rowNum); // Asserts that expected number of records were consumed from the files. HdfsTopicPartitionOffsetMetadata testConsumerTopic013 = new HdfsTopicPartitionOffsetMetadata( @@ -121,14 +121,14 @@ public void testReadAllRecords() { hdfsUri + "opt/teragrep/cfe_39/srv/testConsumerTopic/0.13", 0 ); - AvroReader avroReader2 = new AvroReader(fs, testConsumerTopic013); - avroReader2.open(); - while (avroReader2.next()) { - SyslogRecord syslogRecord = avroReader2.get(); + AvroReadImpl avroReadImpl2 = new AvroReadImpl(fs, testConsumerTopic013); + avroReadImpl2.open(); + while (avroReadImpl2.next()) { + SyslogRecord syslogRecord = avroReadImpl2.get(); Assertions.assertEquals(rowNum, syslogRecord.getOffset()); // Checks offsets of the consumed records which should range from 10 to 13. rowNum++; } - avroReader2.close(); + avroReadImpl2.close(); Assertions.assertEquals(14, rowNum); // Asserts that expected number of records were consumed from the files. }); } From 7c5ee1b91ee2a3412d65a21baac93cc785d33fe2 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Fri, 20 Sep 2024 11:33:37 +0300 Subject: [PATCH 20/62] Added logging --- .../com/teragrep/pth_06/FileSystemFactoryImpl.java | 7 +++++++ .../task/HdfsMicroBatchInputPartitionReader.java | 14 ++++++++++++-- .../teragrep/pth_06/task/hdfs/AvroReadImpl.java | 11 +++++++++++ .../pth_06/task/hdfs/HdfsRecordConversionImpl.java | 4 ++++ 4 files changed, 34 insertions(+), 2 deletions(-) diff --git a/src/main/java/com/teragrep/pth_06/FileSystemFactoryImpl.java b/src/main/java/com/teragrep/pth_06/FileSystemFactoryImpl.java index c31493cb..6433cc52 100644 --- a/src/main/java/com/teragrep/pth_06/FileSystemFactoryImpl.java +++ b/src/main/java/com/teragrep/pth_06/FileSystemFactoryImpl.java @@ -51,12 +51,16 @@ import org.apache.hadoop.fs.LocalFileSystem; 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; public final class FileSystemFactoryImpl implements FileSystemFactory { + final Logger LOGGER = LoggerFactory.getLogger(FileSystemFactoryImpl.class); + private final String kerberosAuthentication; private final String hdfsUri; private final String kerberosRealm; @@ -120,6 +124,7 @@ public FileSystem create(boolean initializeUGI) throws IOException { FileSystem fs; if ("kerberos".equals(kerberosAuthentication)) { // Code for initializing the FileSystem with kerberos. + LOGGER.debug("Kerberos authentication is enabled, using secure authentication for FileSystem."); // set kerberos host and realm System.setProperty("java.security.krb5.realm", kerberosRealm); @@ -146,6 +151,7 @@ public FileSystem create(boolean initializeUGI) throws IOException { conf.set("dfs.encrypt.data.transfer.cipher.suites", hdfsCipherSuites); if (initializeUGI) { + LOGGER.debug("Initializing UserGroupInformation for Kerberos"); UserGroupInformation.setConfiguration(conf); UserGroupInformation.loginUserFromKeytab(kerberosKeytabUser, kerberosKeytabPath); } @@ -155,6 +161,7 @@ public FileSystem create(boolean initializeUGI) throws IOException { } else { // Code for initializing the FileSystem without kerberos. + LOGGER.warn("Kerberos authentication is not enabled, using insecure authentication for FileSystem."); // ====== Init HDFS File System Object Configuration conf = new Configuration(); diff --git a/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReader.java b/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReader.java index 76e46e04..076da672 100644 --- a/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReader.java +++ b/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReader.java @@ -51,11 +51,15 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.connector.read.PartitionReader; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.IOException; import java.util.LinkedList; -public class HdfsMicroBatchInputPartitionReader implements PartitionReader { +public final class HdfsMicroBatchInputPartitionReader implements PartitionReader { + + final Logger LOGGER = LoggerFactory.getLogger(HdfsMicroBatchInputPartitionReader.class); private final LinkedList taskObjectList; private final FileSystem fs; @@ -134,16 +138,22 @@ public boolean next() throws IOException { } } } + LOGGER.debug("next rv: <{}>", rv); return rv; } @Override public InternalRow get() { - return hdfsRecordConversionImpl.get(); + InternalRow rv = hdfsRecordConversionImpl.get(); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug("get(): <{}>", rv.getLong(7)); + } + return rv; } @Override public void close() throws IOException { + LOGGER.debug("HdfsMicroBatchInputPartitionReader.close"); if (!hdfsRecordConversionImpl.isStub()) { hdfsRecordConversionImpl.close(); } diff --git a/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReadImpl.java b/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReadImpl.java index 1312f441..dc425bcb 100644 --- a/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReadImpl.java +++ b/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReadImpl.java @@ -52,12 +52,16 @@ import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.IOException; // This class will allow reading the contents of the avro-files that are using SyslogRecord schema from hdfs. public final class AvroReadImpl implements AvroRead { + final Logger LOGGER = LoggerFactory.getLogger(AvroReadImpl.class); + private final FileSystem fs; private final HdfsTopicPartitionOffsetMetadata hdfsTopicPartitionOffsetMetadata; private DataFileStream reader; @@ -71,6 +75,13 @@ public AvroReadImpl(FileSystem fs, HdfsTopicPartitionOffsetMetadata hdfsTopicPar @Override public void open() throws IOException { Path hdfsreadpath = new Path(hdfsTopicPartitionOffsetMetadata.hdfsFilePath); + if (LOGGER.isDebugEnabled()) { + LOGGER + .debug( + "Attempting to open file <{}> belonging to topic <{}>", hdfsreadpath.getName(), + hdfsTopicPartitionOffsetMetadata.topicPartition.topic() + ); + } FSDataInputStream inputStream = fs.open(hdfsreadpath); reader = new DataFileStream<>(inputStream, new SpecificDatumReader<>(SyslogRecord.class)); } diff --git a/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversionImpl.java b/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversionImpl.java index 0e8a396d..73e757b4 100644 --- a/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversionImpl.java +++ b/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversionImpl.java @@ -52,12 +52,16 @@ import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter; import org.apache.spark.unsafe.types.UTF8String; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.IOException; // This class will read the records from avro-files fetched from HDFS with the help of AvroReadImpl and convert them to InternalRow used by pth_06. public final class HdfsRecordConversionImpl implements HdfsRecordConversion { + final Logger LOGGER = LoggerFactory.getLogger(HdfsRecordConversionImpl.class); + private final boolean stub; private final AvroReadImpl avroReadImpl; private final UnsafeRowWriter rowWriter; From f8d85d492a9e9eb907ddcf4a09816df915659683 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Fri, 20 Sep 2024 12:16:18 +0300 Subject: [PATCH 21/62] Added new test to HdfsMicroBatchInputPartitionReaderTest --- ...dfsMicroBatchInputPartitionReaderTest.java | 41 +++++++++++++++++++ 1 file changed, 41 insertions(+) diff --git a/src/test/java/com/teragrep/pth_06/task/hdfs/HdfsMicroBatchInputPartitionReaderTest.java b/src/test/java/com/teragrep/pth_06/task/hdfs/HdfsMicroBatchInputPartitionReaderTest.java index 57e687f9..d3dbaf2a 100644 --- a/src/test/java/com/teragrep/pth_06/task/hdfs/HdfsMicroBatchInputPartitionReaderTest.java +++ b/src/test/java/com/teragrep/pth_06/task/hdfs/HdfsMicroBatchInputPartitionReaderTest.java @@ -198,4 +198,45 @@ public void testHdfsConsumer1FileAlt() { } + @Test + public void testCutoffEpoch() { + assertDoesNotThrow(() -> { + // create task object list + LinkedList taskObjectList = new LinkedList<>(); + // Add taskObjects to the taskObjectList according to what files are stored in minicluster during setup. + taskObjectList + .add(new HdfsTopicPartitionOffsetMetadata(new TopicPartition("testConsumerTopic", 0), 9, hdfsUri + "opt/teragrep/cfe_39/srv/testConsumerTopic/0.9", 0)); + taskObjectList + .add(new HdfsTopicPartitionOffsetMetadata(new TopicPartition("testConsumerTopic", 0), 13, hdfsUri + "opt/teragrep/cfe_39/srv/testConsumerTopic/0.13", 0)); + + HdfsMicroBatchInputPartitionReader hdfsMicroBatchInputPartitionReader = new HdfsMicroBatchInputPartitionReader( + 1650872090804001L, // Offset 0 has timestamp of 1650872090804000L + "", + hdfsUri, + hdfsPath, + "", + "", + "", + "", + "", + "", + "", + "", + "", + "", + taskObjectList + ); + + // read through the files in HDFS + long rowNum = 1L; // Offset 0 has timestamp of 1650872090804000L, which means it is filtered out by the time based inclusion. + while (hdfsMicroBatchInputPartitionReader.next()) { + InternalRow internalRow = hdfsMicroBatchInputPartitionReader.get(); + Assertions.assertEquals(rowNum, internalRow.getLong(7)); // Checks offsets of the consumed records which should range from 1 to 13. + rowNum++; + } + hdfsMicroBatchInputPartitionReader.close(); + Assertions.assertEquals(14, rowNum); + }); + } + } From 0952390e0e04aefddc01ad239ea005791eb43323 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Fri, 20 Sep 2024 13:44:45 +0300 Subject: [PATCH 22/62] Implemented FileSystemFactoryTest.java --- .../pth_06/FileSystemFactoryTest.java | 118 ++++++++++++++++++ 1 file changed, 118 insertions(+) create mode 100644 src/test/java/com/teragrep/pth_06/FileSystemFactoryTest.java diff --git a/src/test/java/com/teragrep/pth_06/FileSystemFactoryTest.java b/src/test/java/com/teragrep/pth_06/FileSystemFactoryTest.java new file mode 100644 index 00000000..11b6af0c --- /dev/null +++ b/src/test/java/com/teragrep/pth_06/FileSystemFactoryTest.java @@ -0,0 +1,118 @@ +/* + * Teragrep Archive Datasource (pth_06) + * 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.pth_06; + +import com.teragrep.pth_06.planner.MockHDFS; +import org.apache.hadoop.fs.FileSystem; +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 java.io.IOException; + +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; + +public class FileSystemFactoryTest { + + private final String hdfsPath = "hdfs:///opt/teragrep/cfe_39/srv/"; + private String hdfsUri; // Can only be defined after starting the mock hdfs. + private final MockHDFS mockHDFS = new MockHDFS(hdfsPath); + + @BeforeEach + public void setUp() throws IOException, InterruptedException { + hdfsUri = mockHDFS.startMiniCluster(true); + } + + @AfterEach + public void teardown() { + mockHDFS.teardownMiniCluster(); + } + + @Test + public void createInsecureFileSystemTest() { + assertDoesNotThrow(() -> { + FileSystemFactoryImpl fileSystemFactory = new FileSystemFactoryImpl( + "", + hdfsUri, + "", + "", + "", + "", + "", + "", + "", + "", + "", + "" + ); + FileSystem fs = fileSystemFactory.create(true); + fs.close(); + }); + } + + @Test + public void createFileSystemFailureTest() { + FileSystemFactoryImpl fileSystemFactory = new FileSystemFactoryImpl( + "", + "wrongUri", + "", + "", + "", + "", + "", + "", + "", + "", + "", + "" + ); + IllegalArgumentException e = Assertions + .assertThrows(IllegalArgumentException.class, () -> fileSystemFactory.create(true)); + Assertions.assertEquals("java.net.UnknownHostException: wrongUri", e.getMessage()); + } + +} From c8073b0022659f63767c5f793910e00bcbc56a3d Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Fri, 20 Sep 2024 14:21:01 +0300 Subject: [PATCH 23/62] Implemented InstantationKafkaDisabledTest.java --- .../pth_06/InstantationKafkaDisabledTest.java | 458 ++++++++++++++++++ 1 file changed, 458 insertions(+) create mode 100644 src/test/java/com/teragrep/pth_06/InstantationKafkaDisabledTest.java diff --git a/src/test/java/com/teragrep/pth_06/InstantationKafkaDisabledTest.java b/src/test/java/com/teragrep/pth_06/InstantationKafkaDisabledTest.java new file mode 100644 index 00000000..13ab627d --- /dev/null +++ b/src/test/java/com/teragrep/pth_06/InstantationKafkaDisabledTest.java @@ -0,0 +1,458 @@ +/* + * Teragrep Archive Datasource (pth_06) + * 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.pth_06; + +import com.amazonaws.services.s3.AmazonS3; +import com.amazonaws.services.s3.model.Bucket; +import com.cloudbees.syslog.Facility; +import com.cloudbees.syslog.SDElement; +import com.cloudbees.syslog.Severity; +import com.cloudbees.syslog.SyslogMessage; +import com.teragrep.pth_06.planner.MockDBData; +import com.teragrep.pth_06.planner.MockHDFS; +import com.teragrep.pth_06.planner.MockKafkaConsumerFactory; +import com.teragrep.pth_06.task.s3.MockS3; +import com.teragrep.pth_06.task.s3.Pth06S3Client; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.functions; +import org.apache.spark.sql.streaming.StreamingQuery; +import org.apache.spark.sql.streaming.StreamingQueryException; +import org.apache.spark.sql.streaming.Trigger; +import org.jooq.Record11; +import org.jooq.Result; +import org.jooq.types.ULong; +import org.junit.jupiter.api.*; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.sql.Date; +import java.util.Iterator; +import java.util.Map; +import java.util.TreeMap; +import java.util.UUID; +import java.util.concurrent.TimeoutException; +import java.util.zip.GZIPOutputStream; + +@TestInstance(TestInstance.Lifecycle.PER_CLASS) +public class InstantationKafkaDisabledTest { + + private SparkSession spark = null; + + private final String s3endpoint = "http://127.0.0.1:48080"; + private final String s3identity = "s3identity"; + private final String s3credential = "s3credential"; + + private final MockS3 mockS3 = new MockS3(s3endpoint, s3identity, s3credential); + + private final MockDBData mockDBData = new MockDBData(); + + private long expectedRows = 0L; + + private final String hdfsPath = "hdfs:///opt/teragrep/cfe_39/srv/"; + private String hdfsUri; // Can only be defined after starting the mock hdfs. + private final MockHDFS mockHDFS = new MockHDFS(hdfsPath); + + @BeforeAll + public void prepareEnv() throws Exception { + //Logger.getRootLogger().setLevel(Level.ERROR); + //Logger.getLogger("org.apache.spark").setLevel(Level.WARN); + //Logger.getLogger("org.spark-project").setLevel(Level.WARN); + + mockS3.start(); + + // Start mock hdfs with a full set of records from kafka inserted inside. + hdfsUri = mockHDFS.startMiniCluster(true); + + spark = SparkSession + .builder() + .appName("Java Spark SQL basic example") + .master("local[2]") + .config("spark.driver.extraJavaOptions", "-Duser.timezone=EET") + .config("spark.executor.extraJavaOptions", "-Duser.timezone=EET") + .config("spark.sql.session.timeZone", "UTC") + .getOrCreate(); + + //spark.sparkContext().setLogLevel("ERROR"); + + expectedRows = preloadS3Data() + MockKafkaConsumerFactory.getNumRecords(); + /*Files 0.9 and 0.13 are stored to HDFS, 14 records out of 14 will be read from HDFS instead of Kafka.*/ + } + + @Test + public void fullScanDisabledKafkaTest() throws StreamingQueryException, TimeoutException { + // please notice that JAVA_HOME=/usr/lib/jvm/java-1.8.0 mvn clean test -Pdev is required + Dataset df = spark + .readStream() + .format("com.teragrep.pth_06.MockTeragrepDatasource") + .option("archive.enabled", "true") + .option("S3endPoint", s3endpoint) + .option("S3identity", s3identity) + .option("S3credential", s3credential) + .option("DBusername", "mock") + .option("DBpassword", "mock") + .option("DBurl", "mock") + .option("DBstreamdbname", "mock") + .option("DBjournaldbname", "mock") + .option("num_partitions", "1") + .option("queryXML", "") // Only affects HDFS execution in the current test configuration. + // audit information + .option("TeragrepAuditQuery", "index=f17") + .option("TeragrepAuditReason", "test run at fullScanTest()") + .option("TeragrepAuditUser", System.getProperty("user.name")) + // kafka options + .option("kafka.enabled", "false") + .option("kafka.bootstrap.servers", "") + .option("kafka.sasl.mechanism", "") + .option("kafka.security.protocol", "") + .option("kafka.sasl.jaas.config", "") + .option("kafka.useMockKafkaConsumer", "true") + .option("spark.cleaner.referenceTracking.cleanCheckpoints", "true") + // HDFS options + .option("hdfs.enabled", "true") + .option("hdfs.hdfsPath", hdfsPath) + .option("hdfs.hdfsUri", hdfsUri) + .option("hdfs.UseHostName", "false") + .option("hdfs.transferProtection", "test") + .option("hdfs.cipherSuites", "test") + .option("hdfs.useMockHdfsDatabase", "true") + .option("hdfs.krbAuthentication", "false") + .option("hdfs.krbAuthorization", "test") + .option("hdfs.krbPrincipalPattern", "test") + .option("hdfs.krbKdc", "test") + .option("hdfs.krbRealm", "test") + .option("hdfs.krbKeytabUser", "test") + .option("hdfs.krbKeytabPath", "test") + .option("hdfs.useKerberosAutorenewal", "true") + .option("hdfs.includeRecordEpochAndAfter", 0L) // record timestamps are in 1650872090806000 granularity (epochMicros) + .load(); + + Dataset df2 = df.agg(functions.count("*")); + + StreamingQuery streamingQuery = df2 + .writeStream() + .outputMode("complete") + .format("memory") + .trigger(Trigger.ProcessingTime(0)) + .queryName("MockArchiveQuery") + .option("checkpointLocation", "/tmp/checkpoint/" + UUID.randomUUID()) + .option("spark.cleaner.referenceTracking.cleanCheckpoints", "true") + .start(); + + StreamingQuery sq = df.writeStream().foreachBatch((ds, i) -> { + ds.show(false); + }).start(); + sq.processAllAvailable(); + sq.stop(); + sq.awaitTermination(); + + long rowCount = 0; + while (!streamingQuery.awaitTermination(1000)) { + + long resultSize = spark.sqlContext().sql("SELECT * FROM MockArchiveQuery").count(); + if (resultSize > 0) { + rowCount = spark.sqlContext().sql("SELECT * FROM MockArchiveQuery").first().getAs(0); + System.out.println(rowCount); + } + if ( + streamingQuery.lastProgress() == null + || streamingQuery.status().message().equals("Initializing sources") + ) { + // query has not started + } + else if (streamingQuery.lastProgress().sources().length != 0) { + if (isArchiveDone(streamingQuery)) { + streamingQuery.stop(); + } + } + } + Assertions.assertEquals(expectedRows, rowCount); + } + + @Test + public void fullScanDisabledHdfsAndKafkaTest() throws StreamingQueryException, TimeoutException { + // please notice that JAVA_HOME=/usr/lib/jvm/java-1.8.0 mvn clean test -Pdev is required + Dataset df = spark + .readStream() + .format("com.teragrep.pth_06.MockTeragrepDatasource") + .option("archive.enabled", "true") + .option("S3endPoint", s3endpoint) + .option("S3identity", s3identity) + .option("S3credential", s3credential) + .option("DBusername", "mock") + .option("DBpassword", "mock") + .option("DBurl", "mock") + .option("DBstreamdbname", "mock") + .option("DBjournaldbname", "mock") + .option("num_partitions", "1") + .option("queryXML", "") // Only affects HDFS execution in the current test configuration. + // audit information + .option("TeragrepAuditQuery", "index=f17") + .option("TeragrepAuditReason", "test run at fullScanTest()") + .option("TeragrepAuditUser", System.getProperty("user.name")) + // kafka options + .option("kafka.enabled", "false") + .option("kafka.bootstrap.servers", "") + .option("kafka.sasl.mechanism", "") + .option("kafka.security.protocol", "") + .option("kafka.sasl.jaas.config", "") + .option("kafka.useMockKafkaConsumer", "true") + .option("spark.cleaner.referenceTracking.cleanCheckpoints", "true") + // HDFS options + .option("hdfs.enabled", "false") + .option("hdfs.hdfsPath", hdfsPath) + .option("hdfs.hdfsUri", hdfsUri) + .option("hdfs.UseHostName", "false") + .option("hdfs.transferProtection", "test") + .option("hdfs.cipherSuites", "test") + .option("hdfs.useMockHdfsDatabase", "true") + .option("hdfs.krbAuthentication", "false") + .option("hdfs.krbAuthorization", "test") + .option("hdfs.krbPrincipalPattern", "test") + .option("hdfs.krbKdc", "test") + .option("hdfs.krbRealm", "test") + .option("hdfs.krbKeytabUser", "test") + .option("hdfs.krbKeytabPath", "test") + .option("hdfs.useKerberosAutorenewal", "true") + .option("hdfs.includeRecordEpochAndAfter", 0L) // record timestamps are in 1650872090806000 granularity (epochMicros) + .load(); + + Dataset df2 = df.agg(functions.count("*")); + + StreamingQuery streamingQuery = df2 + .writeStream() + .outputMode("complete") + .format("memory") + .trigger(Trigger.ProcessingTime(0)) + .queryName("MockArchiveQuery") + .option("checkpointLocation", "/tmp/checkpoint/" + UUID.randomUUID()) + .option("spark.cleaner.referenceTracking.cleanCheckpoints", "true") + .start(); + + StreamingQuery sq = df.writeStream().foreachBatch((ds, i) -> { + ds.show(false); + }).start(); + sq.processAllAvailable(); + sq.stop(); + sq.awaitTermination(); + + long rowCount = 0; + while (!streamingQuery.awaitTermination(1000)) { + + long resultSize = spark.sqlContext().sql("SELECT * FROM MockArchiveQuery").count(); + if (resultSize > 0) { + rowCount = spark.sqlContext().sql("SELECT * FROM MockArchiveQuery").first().getAs(0); + System.out.println(rowCount); + } + if ( + streamingQuery.lastProgress() == null + || streamingQuery.status().message().equals("Initializing sources") + ) { + // query has not started + } + else if (streamingQuery.lastProgress().sources().length != 0) { + if (isArchiveDone(streamingQuery)) { + streamingQuery.stop(); + } + } + } + Assertions.assertEquals(expectedRows - MockKafkaConsumerFactory.getNumRecords(), rowCount); + } + + private boolean isArchiveDone(StreamingQuery outQ) { + Boolean archiveDone = true; + for (int i = 0; i < outQ.lastProgress().sources().length; i++) { + String startOffset = outQ.lastProgress().sources()[i].startOffset(); + String endOffset = outQ.lastProgress().sources()[i].endOffset(); + String description = outQ.lastProgress().sources()[i].description(); + + if (description != null && !description.startsWith("com.teragrep.pth_06.ArchiveMicroStreamReader@")) { + // ignore others than archive + continue; + } + + if (startOffset != null) { + if (!startOffset.equalsIgnoreCase(endOffset)) { + archiveDone = false; + } + } + else { + archiveDone = false; + } + } + return archiveDone; + } + + @AfterAll + public void decommissionEnv() throws Exception { + mockS3.stop(); + mockHDFS.teardownMiniCluster(); + } + + private long preloadS3Data() throws IOException { + long rows = 0L; + AmazonS3 amazonS3 = new Pth06S3Client(s3endpoint, s3identity, s3credential).build(); + + TreeMap>> virtualDatabaseMap = mockDBData + .getVirtualDatabaseMap(); + + for ( + Map.Entry>> entry : virtualDatabaseMap + .entrySet() + ) { + Iterator> it = entry + .getValue() + .iterator(); + while (it.hasNext()) { + // id, directory, stream, host, logtag, logdate, bucket, path, logtime, filesize + Record11 record10 = it + .next(); + Long id = record10.get(0, ULong.class).longValue(); + String directory = record10.get(1, String.class); + String stream = record10.get(2, String.class); + String host = record10.get(3, String.class); + String logtag = record10.get(4, String.class); + Date logdate = record10.get(5, Date.class); + String bucket = record10.get(6, String.class); + String path = record10.get(7, String.class); + Long logtime = record10.get(8, ULong.class).longValue(); + Long filesize = record10.get(9, ULong.class).longValue(); + + String recordAsJson = record10.formatJSON(); + + // <46>1 2010-01-01T12:34:56.123456+02:00 hostname.domain.tld pstats - - + SyslogMessage syslog = new SyslogMessage(); + syslog = syslog + .withFacility(Facility.USER) + .withSeverity(Severity.WARNING) + .withTimestamp(logtime) + .withHostname(host) + .withAppName(logtag) + .withMsg(recordAsJson); + + // [event_id@48577 hostname="hostname.domain.tld" uuid="" unixtime="" id_source="source"] + + SDElement event_id_48577 = new SDElement("event_id@48577") + .addSDParam("hostname", host) + .addSDParam("uuid", UUID.randomUUID().toString()) + .addSDParam("source", "source") + .addSDParam("unixtime", Long.toString(System.currentTimeMillis())); + + syslog = syslog.withSDElement(event_id_48577); + + // [event_format@48577 original_format="rfc5424"] + + SDElement event_format_48577 = new SDElement("event_id@48577").addSDParam("original_format", "rfc5424"); + + syslog = syslog.withSDElement(event_format_48577); + + // [event_node_relay@48577 hostname="relay.domain.tld" source="hostname.domain.tld" source_module="imudp"] + + SDElement event_node_relay_48577 = new SDElement("event_node_relay@48577") + .addSDParam("hostname", "relay.domain.tld") + .addSDParam("source", host) + .addSDParam("source_module", "imudp"); + + syslog = syslog.withSDElement(event_node_relay_48577); + + // [event_version@48577 major="2" minor="2" hostname="relay.domain.tld" version_source="relay"] + + SDElement event_version_48577 = new SDElement("event_version@48577") + .addSDParam("major", "2") + .addSDParam("minor", "2") + .addSDParam("hostname", "relay.domain.tld") + .addSDParam("version_source", "relay"); + + syslog = syslog.withSDElement(event_version_48577); + + // [event_node_router@48577 source="relay.domain.tld" source_module="imrelp" hostname="router.domain.tld"] + + SDElement event_node_router_48577 = new SDElement("event_node_router@48577") + .addSDParam("source", "relay.domain.tld") + .addSDParam("source_module", "imrelp") + .addSDParam("hostname", "router.domain.tld"); + + syslog = syslog.withSDElement(event_node_router_48577); + + // [origin@48577 hostname="original.hostname.domain.tld"] + + SDElement origin_48577 = new SDElement("origin@48577") + .addSDParam("hostname", "original.hostname.domain.tld"); + syslog = syslog.withSDElement(origin_48577); + + // check if this bucket exists + boolean bucketExists = false; + for (Bucket existingBucket : amazonS3.listBuckets()) { + if (existingBucket.getName().equals(bucket)) { + bucketExists = true; + break; + } + } + if (!bucketExists) { + amazonS3.createBucket(bucket); + } + + // compress the message + String syslogMessage = syslog.toRfc5424SyslogMessage(); + + ByteArrayOutputStream outStream = new ByteArrayOutputStream(syslogMessage.length()); + GZIPOutputStream gzip = new GZIPOutputStream(outStream); + gzip.write(syslogMessage.getBytes()); + gzip.close(); + + ByteArrayInputStream inStream = new ByteArrayInputStream(outStream.toByteArray()); + + // upload as file + amazonS3.putObject(bucket, path, inStream, null); + rows++; + } + + } + return rows; + } +} From 77ba9e0f7971b66be239e453b5763944fc3d640b Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Fri, 20 Sep 2024 14:21:19 +0300 Subject: [PATCH 24/62] Improved comments --- src/test/java/com/teragrep/pth_06/InstantiationTest.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/test/java/com/teragrep/pth_06/InstantiationTest.java b/src/test/java/com/teragrep/pth_06/InstantiationTest.java index 8af39876..9fb7104e 100644 --- a/src/test/java/com/teragrep/pth_06/InstantiationTest.java +++ b/src/test/java/com/teragrep/pth_06/InstantiationTest.java @@ -109,7 +109,7 @@ public void prepareEnv() throws Exception { mockS3.start(); - // Start mock hdfs here in a similar way that the mockS3 is implemented. + // Start mock hdfs with 10 out of 14 records from kafka inserted inside. hdfsUri = mockHDFS.startMiniCluster(false); spark = SparkSession @@ -124,8 +124,7 @@ public void prepareEnv() throws Exception { //spark.sparkContext().setLogLevel("ERROR"); expectedRows = preloadS3Data() + MockKafkaConsumerFactory.getNumRecords(); - /*If file 0.9 is stored to HDFS, 10 records out of 14 will be read from HDFS instead of Kafka. - If files 0.9 and 0.13 are stored to HDFS, all 14 records will be read from HDFS instead of Kafka.*/ + /*File 0.9 is stored to HDFS, 10 records out of 14 will be read from HDFS instead of Kafka.*/ } @Test From d217b08892f47a2dfce1261e98ca9ae35864f91e Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Mon, 14 Oct 2024 15:48:42 +0300 Subject: [PATCH 25/62] Removed obsolete comments. --- .../java/com/teragrep/pth_06/planner/offset/HdfsOffset.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/main/java/com/teragrep/pth_06/planner/offset/HdfsOffset.java b/src/main/java/com/teragrep/pth_06/planner/offset/HdfsOffset.java index e7b41425..f871a71b 100644 --- a/src/main/java/com/teragrep/pth_06/planner/offset/HdfsOffset.java +++ b/src/main/java/com/teragrep/pth_06/planner/offset/HdfsOffset.java @@ -60,8 +60,6 @@ public class HdfsOffset extends Offset implements Serializable { - // TODO: Implement everything that is needed for tracking the offsets for HDFS datasource. - private static final Type mapType = new TypeToken>() { }.getType(); private final Map serializedHdfsOffset; @@ -74,7 +72,6 @@ public HdfsOffset(Map offset) { } } - // TODO: Check if these methods originating from KafkaOffset can be implemented as-is or not. public HdfsOffset(String s) { Gson gson = new Gson(); serializedHdfsOffset = gson.fromJson(s, mapType); From bae696934f26f15bbd2de92764d764cb58f13375 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Tue, 15 Oct 2024 12:38:44 +0300 Subject: [PATCH 26/62] Converted private static literal mapType to a local variable. --- .../java/com/teragrep/pth_06/planner/offset/HdfsOffset.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/src/main/java/com/teragrep/pth_06/planner/offset/HdfsOffset.java b/src/main/java/com/teragrep/pth_06/planner/offset/HdfsOffset.java index f871a71b..34a0eac2 100644 --- a/src/main/java/com/teragrep/pth_06/planner/offset/HdfsOffset.java +++ b/src/main/java/com/teragrep/pth_06/planner/offset/HdfsOffset.java @@ -51,7 +51,6 @@ import org.apache.spark.sql.connector.read.streaming.Offset; import java.io.Serializable; -import java.lang.reflect.Type; import java.util.HashMap; import java.util.Map; @@ -60,8 +59,6 @@ public class HdfsOffset extends Offset implements Serializable { - private static final Type mapType = new TypeToken>() { - }.getType(); private final Map serializedHdfsOffset; public HdfsOffset(Map offset) { @@ -74,7 +71,8 @@ public HdfsOffset(Map offset) { public HdfsOffset(String s) { Gson gson = new Gson(); - serializedHdfsOffset = gson.fromJson(s, mapType); + serializedHdfsOffset = gson.fromJson(s, new TypeToken>() { + }.getType()); } public Map getOffsetMap() { From b6b8741d7da493a9fb147a5f1326f5441009feca Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Tue, 15 Oct 2024 13:33:16 +0300 Subject: [PATCH 27/62] Renamed create() method to fileSystem() in FileSystemFactory interface. --- src/main/java/com/teragrep/pth_06/FileSystemFactory.java | 2 +- src/main/java/com/teragrep/pth_06/FileSystemFactoryImpl.java | 2 +- src/main/java/com/teragrep/pth_06/planner/HdfsDBClient.java | 2 +- .../pth_06/task/HdfsMicroBatchInputPartitionReader.java | 2 +- src/test/java/com/teragrep/pth_06/FileSystemFactoryTest.java | 4 ++-- 5 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/main/java/com/teragrep/pth_06/FileSystemFactory.java b/src/main/java/com/teragrep/pth_06/FileSystemFactory.java index 398736af..d6f2680d 100644 --- a/src/main/java/com/teragrep/pth_06/FileSystemFactory.java +++ b/src/main/java/com/teragrep/pth_06/FileSystemFactory.java @@ -51,6 +51,6 @@ public interface FileSystemFactory { - FileSystem create(boolean initializeUGI) throws IOException; + FileSystem fileSystem(boolean initializeUGI) throws IOException; } diff --git a/src/main/java/com/teragrep/pth_06/FileSystemFactoryImpl.java b/src/main/java/com/teragrep/pth_06/FileSystemFactoryImpl.java index 6433cc52..f12ca9d6 100644 --- a/src/main/java/com/teragrep/pth_06/FileSystemFactoryImpl.java +++ b/src/main/java/com/teragrep/pth_06/FileSystemFactoryImpl.java @@ -120,7 +120,7 @@ public FileSystemFactoryImpl( } @Override - public FileSystem create(boolean initializeUGI) throws IOException { + public FileSystem fileSystem(boolean initializeUGI) throws IOException { FileSystem fs; if ("kerberos".equals(kerberosAuthentication)) { // Code for initializing the FileSystem with kerberos. diff --git a/src/main/java/com/teragrep/pth_06/planner/HdfsDBClient.java b/src/main/java/com/teragrep/pth_06/planner/HdfsDBClient.java index d265022d..86296f38 100644 --- a/src/main/java/com/teragrep/pth_06/planner/HdfsDBClient.java +++ b/src/main/java/com/teragrep/pth_06/planner/HdfsDBClient.java @@ -69,7 +69,7 @@ public HdfsDBClient(Config config, String topicsRegexString) throws IOException this.topicsRegexString = topicsRegexString; this.ignoreBeforeEpoch = config.hdfsConfig.includeFileEpochAndAfter; // Defines the minimum time window / cutoff epoch for making sure that the files which metadata is fetched by the planner are not mistakenly deleted by cfe_39 pruning before tasker can process the records inside. path = config.hdfsConfig.hdfsPath; - fs = new FileSystemFactoryImpl(config.hdfsConfig).create(true); + fs = new FileSystemFactoryImpl(config.hdfsConfig).fileSystem(true); } // this queries and pulls the distinct file metadata values to the partitionList according to the given query conditions (condition only applies to topic names in planner side). diff --git a/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReader.java b/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReader.java index 076da672..86b669b4 100644 --- a/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReader.java +++ b/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReader.java @@ -100,7 +100,7 @@ public HdfsMicroBatchInputPartitionReader( hdfsCipherSuites, kerberosKeytabUser, kerberosKeytabPath - ).create(false); + ).fileSystem(false); this.hdfsRecordConversionImpl = new HdfsRecordConversionImpl(fs); // stub } diff --git a/src/test/java/com/teragrep/pth_06/FileSystemFactoryTest.java b/src/test/java/com/teragrep/pth_06/FileSystemFactoryTest.java index 11b6af0c..9e6a2555 100644 --- a/src/test/java/com/teragrep/pth_06/FileSystemFactoryTest.java +++ b/src/test/java/com/teragrep/pth_06/FileSystemFactoryTest.java @@ -89,7 +89,7 @@ public void createInsecureFileSystemTest() { "", "" ); - FileSystem fs = fileSystemFactory.create(true); + FileSystem fs = fileSystemFactory.fileSystem(true); fs.close(); }); } @@ -111,7 +111,7 @@ public void createFileSystemFailureTest() { "" ); IllegalArgumentException e = Assertions - .assertThrows(IllegalArgumentException.class, () -> fileSystemFactory.create(true)); + .assertThrows(IllegalArgumentException.class, () -> fileSystemFactory.fileSystem(true)); Assertions.assertEquals("java.net.UnknownHostException: wrongUri", e.getMessage()); } From b7e55086a7c3fb5bebb930e0923c015e0f59ce86 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Tue, 15 Oct 2024 15:11:34 +0300 Subject: [PATCH 28/62] Renamed HdfsTopicPartitionOffsetMetadata.java to HdfsFileMetadata.java. --- .../pth_06/ArchiveMicroStreamReader.java | 4 +- ...setMetadata.java => HdfsFileMetadata.java} | 14 ++---- .../teragrep/pth_06/planner/HdfsDBClient.java | 10 ++-- .../teragrep/pth_06/planner/HdfsQuery.java | 7 +-- .../pth_06/planner/HdfsQueryProcessor.java | 46 ++++++------------- .../teragrep/pth_06/scheduler/BatchSlice.java | 22 ++++----- .../scheduler/HdfsBatchSliceCollection.java | 7 ++- .../task/HdfsMicroBatchInputPartition.java | 9 ++-- .../HdfsMicroBatchInputPartitionReader.java | 6 +-- .../pth_06/task/hdfs/AvroReadImpl.java | 12 ++--- .../task/hdfs/HdfsRecordConversionImpl.java | 16 +++---- .../pth_06/task/hdfs/AvroReadImplTest.java | 8 ++-- ...dfsMicroBatchInputPartitionReaderTest.java | 22 ++++----- 13 files changed, 74 insertions(+), 109 deletions(-) rename src/main/java/com/teragrep/pth_06/{HdfsTopicPartitionOffsetMetadata.java => HdfsFileMetadata.java} (83%) diff --git a/src/main/java/com/teragrep/pth_06/ArchiveMicroStreamReader.java b/src/main/java/com/teragrep/pth_06/ArchiveMicroStreamReader.java index a6c3c09b..7151d2dc 100644 --- a/src/main/java/com/teragrep/pth_06/ArchiveMicroStreamReader.java +++ b/src/main/java/com/teragrep/pth_06/ArchiveMicroStreamReader.java @@ -357,10 +357,10 @@ public InputPartition[] planInputPartitions(Offset start, Offset end) { } // HDFS tasks - LinkedList hdfsTaskList = new LinkedList<>(); + LinkedList hdfsTaskList = new LinkedList<>(); for (BatchSlice batchSlice : taskObjectList) { if (batchSlice.type.equals(BatchSlice.Type.HDFS)) { - hdfsTaskList.add(batchSlice.hdfsTopicPartitionOffsetMetadata); + hdfsTaskList.add(batchSlice.hdfsFileMetadata); } } diff --git a/src/main/java/com/teragrep/pth_06/HdfsTopicPartitionOffsetMetadata.java b/src/main/java/com/teragrep/pth_06/HdfsFileMetadata.java similarity index 83% rename from src/main/java/com/teragrep/pth_06/HdfsTopicPartitionOffsetMetadata.java rename to src/main/java/com/teragrep/pth_06/HdfsFileMetadata.java index 31fea2e4..b99982f3 100644 --- a/src/main/java/com/teragrep/pth_06/HdfsTopicPartitionOffsetMetadata.java +++ b/src/main/java/com/teragrep/pth_06/HdfsFileMetadata.java @@ -50,20 +50,14 @@ import java.io.Serializable; // Class for holding serializable metadata of HDFS files containing kafka records. -// Maybe change the class name to something more appropriate. ie. HdfsFileMetadata -public class HdfsTopicPartitionOffsetMetadata implements Serializable { +public class HdfsFileMetadata implements Serializable { public final TopicPartition topicPartition; // Represents the Kafka topic partition which records the file contains. public final long endOffset; // Represents the offset of the record that was last added to the file. public final String hdfsFilePath; // Represents the file path where the file resides in HDFS. public final long hdfsFileSize; // Represents the size of the file in HDFS. Used for scheduling the batch slice. - public HdfsTopicPartitionOffsetMetadata( - TopicPartition topicPartition, - long offset, - String filePath, - long fileSize - ) { + public HdfsFileMetadata(TopicPartition topicPartition, long offset, String filePath, long fileSize) { this.topicPartition = topicPartition; this.endOffset = offset; this.hdfsFilePath = filePath; @@ -72,7 +66,7 @@ public HdfsTopicPartitionOffsetMetadata( @Override public String toString() { - return "HdfsTopicPartitionOffsetMetadata{" + "topicPartition=" + topicPartition + ", endOffset=" + endOffset - + ", hdfsFilePath=" + hdfsFilePath + ", hdfsFileSize=" + hdfsFileSize + '}'; + return "HdfsFileMetadata{" + "topicPartition=" + topicPartition + ", endOffset=" + endOffset + ", hdfsFilePath=" + + hdfsFilePath + ", hdfsFileSize=" + hdfsFileSize + '}'; } } diff --git a/src/main/java/com/teragrep/pth_06/planner/HdfsDBClient.java b/src/main/java/com/teragrep/pth_06/planner/HdfsDBClient.java index 86296f38..2a4fb7a7 100644 --- a/src/main/java/com/teragrep/pth_06/planner/HdfsDBClient.java +++ b/src/main/java/com/teragrep/pth_06/planner/HdfsDBClient.java @@ -46,8 +46,8 @@ package com.teragrep.pth_06.planner; import com.teragrep.pth_06.FileSystemFactoryImpl; +import com.teragrep.pth_06.HdfsFileMetadata; import com.teragrep.pth_06.config.Config; -import com.teragrep.pth_06.HdfsTopicPartitionOffsetMetadata; import org.apache.hadoop.fs.*; import org.apache.kafka.common.TopicPartition; import org.slf4j.Logger; @@ -73,8 +73,8 @@ public HdfsDBClient(Config config, String topicsRegexString) throws IOException } // this queries and pulls the distinct file metadata values to the partitionList according to the given query conditions (condition only applies to topic names in planner side). - public LinkedList pullToPartitionList() throws IOException { - LinkedList rv = new LinkedList<>(); + public LinkedList pullToPartitionList() throws IOException { + LinkedList rv = new LinkedList<>(); // path holds the fileSystem path to the directory that holds a collection of other directories, each different directory representing a different topic. FileStatus[] directoryStatuses = fs.listStatus(new Path(path), topicFilter); /*If the path only holds one directory, fileStatuses will only hold one FileStatus object which returns this value when fileStatus.getPath() is called: @@ -91,13 +91,13 @@ public LinkedList pullToPartitionList() throws 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]; - HdfsTopicPartitionOffsetMetadata temp = new HdfsTopicPartitionOffsetMetadata( + HdfsFileMetadata temp = new HdfsFileMetadata( new TopicPartition(topic, Integer.parseInt(partition)), Integer.parseInt(offset), r2.getPath().toString(), r2.getLen() ); - // Add the HdfsTopicPartitionOffsetMetadata object to the rv only if the file's modification timestamp is above ignoreBeforeEpoch. Timestamps are in milliseconds. + // Add the HdfsFileMetadata object to the rv only if the file's modification timestamp is above ignoreBeforeEpoch. Timestamps are in milliseconds. if (r2.getModificationTime() >= ignoreBeforeEpoch) { rv.add(temp); } diff --git a/src/main/java/com/teragrep/pth_06/planner/HdfsQuery.java b/src/main/java/com/teragrep/pth_06/planner/HdfsQuery.java index 711d4d8f..76188484 100644 --- a/src/main/java/com/teragrep/pth_06/planner/HdfsQuery.java +++ b/src/main/java/com/teragrep/pth_06/planner/HdfsQuery.java @@ -46,17 +46,14 @@ package com.teragrep.pth_06.planner; import com.google.gson.JsonArray; -import com.teragrep.pth_06.HdfsTopicPartitionOffsetMetadata; +import com.teragrep.pth_06.HdfsFileMetadata; import com.teragrep.pth_06.planner.offset.HdfsOffset; import java.util.LinkedList; public interface HdfsQuery { - LinkedList processBetweenHdfsFileMetadata( - HdfsOffset startOffset, - HdfsOffset endOffset - ); + LinkedList processBetweenHdfsFileMetadata(HdfsOffset startOffset, HdfsOffset endOffset); void commit(HdfsOffset offset); diff --git a/src/main/java/com/teragrep/pth_06/planner/HdfsQueryProcessor.java b/src/main/java/com/teragrep/pth_06/planner/HdfsQueryProcessor.java index f2288b6f..b52242eb 100644 --- a/src/main/java/com/teragrep/pth_06/planner/HdfsQueryProcessor.java +++ b/src/main/java/com/teragrep/pth_06/planner/HdfsQueryProcessor.java @@ -47,8 +47,8 @@ import com.google.gson.JsonArray; import com.google.gson.JsonObject; +import com.teragrep.pth_06.HdfsFileMetadata; import com.teragrep.pth_06.config.Config; -import com.teragrep.pth_06.HdfsTopicPartitionOffsetMetadata; import com.teragrep.pth_06.planner.offset.HdfsOffset; import com.teragrep.pth_06.planner.walker.HdfsConditionWalker; import org.apache.kafka.common.TopicPartition; @@ -65,7 +65,7 @@ public class HdfsQueryProcessor implements HdfsQuery { private final Logger LOGGER = LoggerFactory.getLogger(HdfsQueryProcessor.class); - private LinkedList topicPartitionList; + private LinkedList topicPartitionList; private final HdfsDBClient hdfsDBClient; private String topicsRegexString; private final Map hdfsOffsetMap; @@ -112,7 +112,7 @@ public HdfsQueryProcessor(Config config) { hdfsOffsetMap = new HashMap<>(); // This parameter is used for generating the new start offsets for the KafkaOffsetPlanner. hdfsOffsetMapToJSON() is used to transfer the parameter using printer. seekToResults(); // This method loads all the available metadata to TopicPartitionList from HDFS. // Create a map that only contains the metadata with the highest offset for every topic partition. - for (HdfsTopicPartitionOffsetMetadata r : topicPartitionList) { + for (HdfsFileMetadata r : topicPartitionList) { long partitionStart = r.endOffset; if (!hdfsOffsetMap.containsKey(r.topicPartition)) { hdfsOffsetMap.put(r.topicPartition, partitionStart + 1); @@ -141,27 +141,16 @@ private void seekToResults() { // Returns all the available HDFS file metadata between the given topic partition offsets. @Override - public LinkedList processBetweenHdfsFileMetadata( - HdfsOffset startOffset, - HdfsOffset endOffset - ) { - LinkedList rv = new LinkedList<>(); + public LinkedList processBetweenHdfsFileMetadata(HdfsOffset startOffset, HdfsOffset endOffset) { + LinkedList rv = new LinkedList<>(); Map endOffsetMap = endOffset.getOffsetMap(); Map startOffsetMap = startOffset.getOffsetMap(); - for (HdfsTopicPartitionOffsetMetadata r : topicPartitionList) { + for (HdfsFileMetadata r : topicPartitionList) { if ( (endOffsetMap.get(r.topicPartition) >= r.endOffset) & (startOffsetMap.get(r.topicPartition) <= r.endOffset) ) { - rv - .add( - new HdfsTopicPartitionOffsetMetadata( - r.topicPartition, - r.endOffset, - r.hdfsFilePath, - r.hdfsFileSize - ) - ); + rv.add(new HdfsFileMetadata(r.topicPartition, r.endOffset, r.hdfsFilePath, r.hdfsFileSize)); } } return rv; @@ -171,19 +160,12 @@ public LinkedList processBetweenHdfsFileMetada @Override public void commit(HdfsOffset offset) { Map offsetMap = offset.getOffsetMap(); - LinkedList newTopicPartitionList = new LinkedList<>(); + LinkedList newTopicPartitionList = new LinkedList<>(); // Generate new topicPartitionList where the metadata with offset values lower than the offset values given as input parameter are filtered out. - for (HdfsTopicPartitionOffsetMetadata r : topicPartitionList) { + for (HdfsFileMetadata r : topicPartitionList) { if (offsetMap.get(r.topicPartition) < r.endOffset) { newTopicPartitionList - .add( - new HdfsTopicPartitionOffsetMetadata( - r.topicPartition, - r.endOffset, - r.hdfsFilePath, - r.hdfsFileSize - ) - ); + .add(new HdfsFileMetadata(r.topicPartition, r.endOffset, r.hdfsFilePath, r.hdfsFileSize)); } } topicPartitionList = newTopicPartitionList; @@ -211,7 +193,7 @@ public JsonArray hdfsOffsetMapToJSON() { public HdfsOffset getBeginningOffsets() { Map startOffset = new HashMap<>(); // Go through the topicPartitionList to generate start offsets. - for (HdfsTopicPartitionOffsetMetadata r : topicPartitionList) { + for (HdfsFileMetadata r : topicPartitionList) { // Generate startOffset // When going through the result, store the topic partition with the lowest offset to the startOffset object. long partitionOffset = r.endOffset; @@ -232,7 +214,7 @@ public HdfsOffset getBeginningOffsets() { public HdfsOffset getInitialEndOffsets() { Map endOffset = new HashMap<>(); // Go through the topicPartitionList to generate end offsets. - for (HdfsTopicPartitionOffsetMetadata r : topicPartitionList) { + for (HdfsFileMetadata r : topicPartitionList) { long partitionOffset = r.endOffset; // Generate endOffset // When going through the result, store the topic partition with the highest offset to the endOffset object. @@ -257,9 +239,9 @@ public HdfsOffset incrementAndGetLatestOffset() { // Initialize the batchSizeLimit object to split the data into appropriate sized batches BatchSizeLimit batchSizeLimit = new BatchSizeLimit(quantumLength * numPartitions, totalObjectCountLimit); // Keep loading more offsets from topicPartitionList until the limit is reached - Iterator iterator = topicPartitionList.iterator(); + Iterator iterator = topicPartitionList.iterator(); while (!batchSizeLimit.isOverLimit() && iterator.hasNext()) { - HdfsTopicPartitionOffsetMetadata r = iterator.next(); + HdfsFileMetadata r = iterator.next(); // When going through the result, store the topic partition with the highest offset to the latestHdfsOffsetMap. if (latestHdfsOffsetMap.get(r.topicPartition) < r.endOffset) { latestHdfsOffsetMap.replace(r.topicPartition, r.endOffset); diff --git a/src/main/java/com/teragrep/pth_06/scheduler/BatchSlice.java b/src/main/java/com/teragrep/pth_06/scheduler/BatchSlice.java index 707c2995..56905171 100644 --- a/src/main/java/com/teragrep/pth_06/scheduler/BatchSlice.java +++ b/src/main/java/com/teragrep/pth_06/scheduler/BatchSlice.java @@ -46,7 +46,7 @@ package com.teragrep.pth_06.scheduler; import com.teragrep.pth_06.ArchiveS3ObjectMetadata; -import com.teragrep.pth_06.HdfsTopicPartitionOffsetMetadata; +import com.teragrep.pth_06.HdfsFileMetadata; import com.teragrep.pth_06.KafkaTopicPartitionOffsetMetadata; import java.io.Serializable; @@ -69,27 +69,27 @@ public enum Type { public final Type type; public final ArchiveS3ObjectMetadata archiveS3ObjectMetadata; public final KafkaTopicPartitionOffsetMetadata kafkaTopicPartitionOffsetMetadata; - public final HdfsTopicPartitionOffsetMetadata hdfsTopicPartitionOffsetMetadata; + public final HdfsFileMetadata hdfsFileMetadata; public BatchSlice(ArchiveS3ObjectMetadata archiveS3ObjectMetadata) { this.type = Type.ARCHIVE; this.archiveS3ObjectMetadata = archiveS3ObjectMetadata; this.kafkaTopicPartitionOffsetMetadata = null; - this.hdfsTopicPartitionOffsetMetadata = null; + this.hdfsFileMetadata = null; } public BatchSlice(KafkaTopicPartitionOffsetMetadata kafkaTopicPartitionOffsetMetadata) { this.type = Type.KAFKA; this.archiveS3ObjectMetadata = null; this.kafkaTopicPartitionOffsetMetadata = kafkaTopicPartitionOffsetMetadata; - this.hdfsTopicPartitionOffsetMetadata = null; + this.hdfsFileMetadata = null; } - public BatchSlice(HdfsTopicPartitionOffsetMetadata hdfsTopicPartitionOffsetMetadata) { + public BatchSlice(HdfsFileMetadata hdfsFileMetadata) { this.type = Type.HDFS; this.archiveS3ObjectMetadata = null; this.kafkaTopicPartitionOffsetMetadata = null; - this.hdfsTopicPartitionOffsetMetadata = hdfsTopicPartitionOffsetMetadata; + this.hdfsFileMetadata = hdfsFileMetadata; } @@ -108,9 +108,9 @@ public long getSize() { // TODO estimate based on offset delta return 1024 * 1024 * 16; case HDFS: - // The files in HDFS have a default size limiter set to them during creation. Use that limit (64000000) or hdfsTopicPartitionOffsetMetadata.getFileSize(). - if (hdfsTopicPartitionOffsetMetadata != null) { - return hdfsTopicPartitionOffsetMetadata.hdfsFileSize; + // The files in HDFS have a default size limiter set to them during creation. Use that limit (64000000) or hdfsFileMetadata.getFileSize(). + if (hdfsFileMetadata != null) { + return hdfsFileMetadata.hdfsFileSize; } else { throw new RuntimeException("Expected HDFS file metadata, instead was null"); @@ -123,7 +123,7 @@ public long getSize() { @Override public String toString() { return "BatchSlice{" + "batchSliceType=" + type + ", archiveS3ObjectMetadata=" + archiveS3ObjectMetadata - + ", kafkaTopicPartitionOffsetMetadata=" + kafkaTopicPartitionOffsetMetadata - + ", hdfsTopicPartitionOffsetMetadata=" + hdfsTopicPartitionOffsetMetadata + '}'; + + ", kafkaTopicPartitionOffsetMetadata=" + kafkaTopicPartitionOffsetMetadata + ", hdfsFileMetadata=" + + hdfsFileMetadata + '}'; } } diff --git a/src/main/java/com/teragrep/pth_06/scheduler/HdfsBatchSliceCollection.java b/src/main/java/com/teragrep/pth_06/scheduler/HdfsBatchSliceCollection.java index f5cbf262..059a7ab1 100644 --- a/src/main/java/com/teragrep/pth_06/scheduler/HdfsBatchSliceCollection.java +++ b/src/main/java/com/teragrep/pth_06/scheduler/HdfsBatchSliceCollection.java @@ -45,7 +45,7 @@ */ package com.teragrep.pth_06.scheduler; -import com.teragrep.pth_06.HdfsTopicPartitionOffsetMetadata; +import com.teragrep.pth_06.HdfsFileMetadata; import com.teragrep.pth_06.planner.HdfsQuery; import com.teragrep.pth_06.planner.offset.DatasourceOffset; import com.teragrep.pth_06.planner.offset.HdfsOffset; @@ -104,9 +104,8 @@ public HdfsBatchSliceCollection processRange(Offset start, Offset end) { LOGGER.debug("processRange(): args: start: " + start + " end: " + end); HdfsOffset hdfsStartOffset = ((DatasourceOffset) start).getHdfsOffset(); HdfsOffset hdfsEndOffset = ((DatasourceOffset) end).getHdfsOffset(); - LinkedList result = hq - .processBetweenHdfsFileMetadata(hdfsStartOffset, hdfsEndOffset); - for (HdfsTopicPartitionOffsetMetadata r : result) { + LinkedList result = hq.processBetweenHdfsFileMetadata(hdfsStartOffset, hdfsEndOffset); + for (HdfsFileMetadata r : result) { this.add(new BatchSlice(r)); } return this; diff --git a/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartition.java b/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartition.java index 01074dcd..e9e7e509 100644 --- a/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartition.java +++ b/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartition.java @@ -45,7 +45,7 @@ */ package com.teragrep.pth_06.task; -import com.teragrep.pth_06.HdfsTopicPartitionOffsetMetadata; +import com.teragrep.pth_06.HdfsFileMetadata; import com.teragrep.pth_06.config.HdfsConfig; import org.apache.spark.sql.connector.read.InputPartition; import org.slf4j.Logger; @@ -57,7 +57,7 @@ public class HdfsMicroBatchInputPartition implements InputPartition { public final Logger LOGGER = LoggerFactory.getLogger(HdfsMicroBatchInputPartition.class); - public final LinkedList taskObjectList; + public final LinkedList taskObjectList; public final long includeRecordEpochAndAfter; // Represents the cutoff epoch which dictates which files should not be fetched from HDFS based on their timestamps. public final String hdfsPath; // Represents the working directory path in HDFS filesystem. @@ -75,10 +75,7 @@ public class HdfsMicroBatchInputPartition implements InputPartition { public final String kerberosKeytabPath; public final String kerberosAutorenewal; - public HdfsMicroBatchInputPartition( - HdfsConfig hdfsConfig, - LinkedList hdfsTaskList - ) { + public HdfsMicroBatchInputPartition(HdfsConfig hdfsConfig, LinkedList hdfsTaskList) { this.taskObjectList = hdfsTaskList; kerberosAutorenewal = hdfsConfig.kerberosAutorenewal; hdfsUri = hdfsConfig.hdfsUri; diff --git a/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReader.java b/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReader.java index 86b669b4..c4f8e36d 100644 --- a/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReader.java +++ b/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReader.java @@ -46,7 +46,7 @@ package com.teragrep.pth_06.task; import com.teragrep.pth_06.FileSystemFactoryImpl; -import com.teragrep.pth_06.HdfsTopicPartitionOffsetMetadata; +import com.teragrep.pth_06.HdfsFileMetadata; import com.teragrep.pth_06.task.hdfs.HdfsRecordConversionImpl; import org.apache.hadoop.fs.FileSystem; import org.apache.spark.sql.catalyst.InternalRow; @@ -61,7 +61,7 @@ public final class HdfsMicroBatchInputPartitionReader implements PartitionReader final Logger LOGGER = LoggerFactory.getLogger(HdfsMicroBatchInputPartitionReader.class); - private final LinkedList taskObjectList; + private final LinkedList taskObjectList; private final FileSystem fs; private final long cutoffEpoch; private HdfsRecordConversionImpl hdfsRecordConversionImpl; @@ -81,7 +81,7 @@ public HdfsMicroBatchInputPartitionReader( String hdfsCipherSuites, String kerberosKeytabUser, String kerberosKeytabPath, - LinkedList taskObjectList + LinkedList taskObjectList ) throws IOException { this.cutoffEpoch = cutoffEpoch; diff --git a/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReadImpl.java b/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReadImpl.java index dc425bcb..53f62132 100644 --- a/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReadImpl.java +++ b/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReadImpl.java @@ -45,7 +45,7 @@ */ package com.teragrep.pth_06.task.hdfs; -import com.teragrep.pth_06.HdfsTopicPartitionOffsetMetadata; +import com.teragrep.pth_06.HdfsFileMetadata; import com.teragrep.pth_06.avro.SyslogRecord; import org.apache.avro.file.DataFileStream; import org.apache.avro.specific.SpecificDatumReader; @@ -63,23 +63,23 @@ public final class AvroReadImpl implements AvroRead { final Logger LOGGER = LoggerFactory.getLogger(AvroReadImpl.class); private final FileSystem fs; - private final HdfsTopicPartitionOffsetMetadata hdfsTopicPartitionOffsetMetadata; + private final HdfsFileMetadata hdfsFileMetadata; private DataFileStream reader; private SyslogRecord currentRecord; - public AvroReadImpl(FileSystem fs, HdfsTopicPartitionOffsetMetadata hdfsTopicPartitionOffsetMetadata) { + public AvroReadImpl(FileSystem fs, HdfsFileMetadata hdfsFileMetadata) { this.fs = fs; - this.hdfsTopicPartitionOffsetMetadata = hdfsTopicPartitionOffsetMetadata; + this.hdfsFileMetadata = hdfsFileMetadata; } @Override public void open() throws IOException { - Path hdfsreadpath = new Path(hdfsTopicPartitionOffsetMetadata.hdfsFilePath); + Path hdfsreadpath = new Path(hdfsFileMetadata.hdfsFilePath); if (LOGGER.isDebugEnabled()) { LOGGER .debug( "Attempting to open file <{}> belonging to topic <{}>", hdfsreadpath.getName(), - hdfsTopicPartitionOffsetMetadata.topicPartition.topic() + hdfsFileMetadata.topicPartition.topic() ); } FSDataInputStream inputStream = fs.open(hdfsreadpath); diff --git a/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversionImpl.java b/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversionImpl.java index 73e757b4..a7e3fccb 100644 --- a/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversionImpl.java +++ b/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversionImpl.java @@ -45,7 +45,7 @@ */ package com.teragrep.pth_06.task.hdfs; -import com.teragrep.pth_06.HdfsTopicPartitionOffsetMetadata; +import com.teragrep.pth_06.HdfsFileMetadata; import com.teragrep.pth_06.avro.SyslogRecord; import org.apache.hadoop.fs.FileSystem; import org.apache.kafka.common.TopicPartition; @@ -68,19 +68,15 @@ public final class HdfsRecordConversionImpl implements HdfsRecordConversion { // Stub object public HdfsRecordConversionImpl(FileSystem fs) { - this(fs, new HdfsTopicPartitionOffsetMetadata(new TopicPartition("", 0), 0, "", 0), true); + this(fs, new HdfsFileMetadata(new TopicPartition("", 0), 0, "", 0), true); } - public HdfsRecordConversionImpl(FileSystem fs, HdfsTopicPartitionOffsetMetadata hdfsTopicPartitionOffsetMetadata) { - this(fs, hdfsTopicPartitionOffsetMetadata, false); + public HdfsRecordConversionImpl(FileSystem fs, HdfsFileMetadata hdfsFileMetadata) { + this(fs, hdfsFileMetadata, false); } - public HdfsRecordConversionImpl( - FileSystem fs, - HdfsTopicPartitionOffsetMetadata hdfsTopicPartitionOffsetMetadata, - Boolean stub - ) { - this.avroReadImpl = new AvroReadImpl(fs, hdfsTopicPartitionOffsetMetadata); + public HdfsRecordConversionImpl(FileSystem fs, HdfsFileMetadata hdfsFileMetadata, Boolean stub) { + this.avroReadImpl = new AvroReadImpl(fs, hdfsFileMetadata); this.stub = stub; this.rowWriter = new UnsafeRowWriter(11); } diff --git a/src/test/java/com/teragrep/pth_06/task/hdfs/AvroReadImplTest.java b/src/test/java/com/teragrep/pth_06/task/hdfs/AvroReadImplTest.java index cdb56785..6a223a63 100644 --- a/src/test/java/com/teragrep/pth_06/task/hdfs/AvroReadImplTest.java +++ b/src/test/java/com/teragrep/pth_06/task/hdfs/AvroReadImplTest.java @@ -45,7 +45,7 @@ */ package com.teragrep.pth_06.task.hdfs; -import com.teragrep.pth_06.HdfsTopicPartitionOffsetMetadata; +import com.teragrep.pth_06.HdfsFileMetadata; import com.teragrep.pth_06.avro.SyslogRecord; import com.teragrep.pth_06.planner.MockHDFS; import org.apache.hadoop.conf.Configuration; @@ -97,8 +97,8 @@ public void testReadAllRecords() { // filesystem for HDFS access is set here FileSystem fs = FileSystem.get(URI.create(hdfsUri), conf); - // build hdfsTopicPartitionOffsetMetadata object - HdfsTopicPartitionOffsetMetadata testConsumerTopic09 = new HdfsTopicPartitionOffsetMetadata( + // build hdfsFileMetadata object + HdfsFileMetadata testConsumerTopic09 = new HdfsFileMetadata( new TopicPartition("testConsumerTopic", 0), 9, hdfsUri + "opt/teragrep/cfe_39/srv/testConsumerTopic/0.9", @@ -115,7 +115,7 @@ public void testReadAllRecords() { avroReadImpl1.close(); Assertions.assertEquals(10, rowNum); // Asserts that expected number of records were consumed from the files. - HdfsTopicPartitionOffsetMetadata testConsumerTopic013 = new HdfsTopicPartitionOffsetMetadata( + HdfsFileMetadata testConsumerTopic013 = new HdfsFileMetadata( new TopicPartition("testConsumerTopic", 0), 13, hdfsUri + "opt/teragrep/cfe_39/srv/testConsumerTopic/0.13", diff --git a/src/test/java/com/teragrep/pth_06/task/hdfs/HdfsMicroBatchInputPartitionReaderTest.java b/src/test/java/com/teragrep/pth_06/task/hdfs/HdfsMicroBatchInputPartitionReaderTest.java index d3dbaf2a..e86fa43b 100644 --- a/src/test/java/com/teragrep/pth_06/task/hdfs/HdfsMicroBatchInputPartitionReaderTest.java +++ b/src/test/java/com/teragrep/pth_06/task/hdfs/HdfsMicroBatchInputPartitionReaderTest.java @@ -45,7 +45,7 @@ */ package com.teragrep.pth_06.task.hdfs; -import com.teragrep.pth_06.HdfsTopicPartitionOffsetMetadata; +import com.teragrep.pth_06.HdfsFileMetadata; import com.teragrep.pth_06.planner.MockHDFS; import com.teragrep.pth_06.task.HdfsMicroBatchInputPartitionReader; import org.apache.kafka.common.TopicPartition; @@ -80,12 +80,12 @@ public void teardown() { public void testHdfsConsumer2Files() { assertDoesNotThrow(() -> { // create task object list - LinkedList taskObjectList = new LinkedList<>(); + LinkedList taskObjectList = new LinkedList<>(); // Add taskObjects to the taskObjectList according to what files are stored in minicluster during setup. taskObjectList - .add(new HdfsTopicPartitionOffsetMetadata(new TopicPartition("testConsumerTopic", 0), 9, hdfsUri + "opt/teragrep/cfe_39/srv/testConsumerTopic/0.9", 0)); + .add(new HdfsFileMetadata(new TopicPartition("testConsumerTopic", 0), 9, hdfsUri + "opt/teragrep/cfe_39/srv/testConsumerTopic/0.9", 0)); taskObjectList - .add(new HdfsTopicPartitionOffsetMetadata(new TopicPartition("testConsumerTopic", 0), 13, hdfsUri + "opt/teragrep/cfe_39/srv/testConsumerTopic/0.13", 0)); + .add(new HdfsFileMetadata(new TopicPartition("testConsumerTopic", 0), 13, hdfsUri + "opt/teragrep/cfe_39/srv/testConsumerTopic/0.13", 0)); HdfsMicroBatchInputPartitionReader hdfsMicroBatchInputPartitionReader = new HdfsMicroBatchInputPartitionReader( 0L, @@ -122,10 +122,10 @@ public void testHdfsConsumer2Files() { public void testHdfsConsumer1File() { assertDoesNotThrow(() -> { // create task object list - LinkedList taskObjectList = new LinkedList<>(); + LinkedList taskObjectList = new LinkedList<>(); // Add only the taskObject related to testConsumerTopic/0.9 file to the taskObjectList. taskObjectList - .add(new HdfsTopicPartitionOffsetMetadata(new TopicPartition("testConsumerTopic", 0), 9, hdfsUri + "opt/teragrep/cfe_39/srv/testConsumerTopic/0.9", 0)); + .add(new HdfsFileMetadata(new TopicPartition("testConsumerTopic", 0), 9, hdfsUri + "opt/teragrep/cfe_39/srv/testConsumerTopic/0.9", 0)); HdfsMicroBatchInputPartitionReader hdfsMicroBatchInputPartitionReader = new HdfsMicroBatchInputPartitionReader( 0L, @@ -162,10 +162,10 @@ public void testHdfsConsumer1File() { public void testHdfsConsumer1FileAlt() { assertDoesNotThrow(() -> { // create task object list - LinkedList taskObjectList = new LinkedList<>(); + LinkedList taskObjectList = new LinkedList<>(); // Add only the taskObject related to testConsumerTopic/0.13 file to the taskObjectList. taskObjectList - .add(new HdfsTopicPartitionOffsetMetadata(new TopicPartition("testConsumerTopic", 0), 13, hdfsUri + "opt/teragrep/cfe_39/srv/testConsumerTopic/0.13", 0)); + .add(new HdfsFileMetadata(new TopicPartition("testConsumerTopic", 0), 13, hdfsUri + "opt/teragrep/cfe_39/srv/testConsumerTopic/0.13", 0)); HdfsMicroBatchInputPartitionReader hdfsMicroBatchInputPartitionReader = new HdfsMicroBatchInputPartitionReader( 0L, @@ -202,12 +202,12 @@ public void testHdfsConsumer1FileAlt() { public void testCutoffEpoch() { assertDoesNotThrow(() -> { // create task object list - LinkedList taskObjectList = new LinkedList<>(); + LinkedList taskObjectList = new LinkedList<>(); // Add taskObjects to the taskObjectList according to what files are stored in minicluster during setup. taskObjectList - .add(new HdfsTopicPartitionOffsetMetadata(new TopicPartition("testConsumerTopic", 0), 9, hdfsUri + "opt/teragrep/cfe_39/srv/testConsumerTopic/0.9", 0)); + .add(new HdfsFileMetadata(new TopicPartition("testConsumerTopic", 0), 9, hdfsUri + "opt/teragrep/cfe_39/srv/testConsumerTopic/0.9", 0)); taskObjectList - .add(new HdfsTopicPartitionOffsetMetadata(new TopicPartition("testConsumerTopic", 0), 13, hdfsUri + "opt/teragrep/cfe_39/srv/testConsumerTopic/0.13", 0)); + .add(new HdfsFileMetadata(new TopicPartition("testConsumerTopic", 0), 13, hdfsUri + "opt/teragrep/cfe_39/srv/testConsumerTopic/0.13", 0)); HdfsMicroBatchInputPartitionReader hdfsMicroBatchInputPartitionReader = new HdfsMicroBatchInputPartitionReader( 1650872090804001L, // Offset 0 has timestamp of 1650872090804000L From f4bfe075404c9bd418c940599026d1167fd2cd3d Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Tue, 15 Oct 2024 15:52:30 +0300 Subject: [PATCH 29/62] Refactored UseHdfsHostname naming usage to useHdfsHostname. --- .../java/com/teragrep/pth_06/FileSystemFactoryImpl.java | 8 ++++---- src/main/java/com/teragrep/pth_06/config/HdfsConfig.java | 6 +++--- .../pth_06/task/HdfsMicroBatchInputPartition.java | 4 ++-- .../pth_06/task/TeragrepPartitionReaderFactory.java | 2 +- 4 files changed, 10 insertions(+), 10 deletions(-) diff --git a/src/main/java/com/teragrep/pth_06/FileSystemFactoryImpl.java b/src/main/java/com/teragrep/pth_06/FileSystemFactoryImpl.java index f12ca9d6..3351d92b 100644 --- a/src/main/java/com/teragrep/pth_06/FileSystemFactoryImpl.java +++ b/src/main/java/com/teragrep/pth_06/FileSystemFactoryImpl.java @@ -67,7 +67,7 @@ public final class FileSystemFactoryImpl implements FileSystemFactory { private final String kerberosKdc; private final String kerberosAuthorization; private final String kerberosAutorenewal; - private final String UseHdfsHostname; + private final String useHdfsHostname; private final String kerberosPrincipalPattern; private final String hdfsTransferProtection; private final String hdfsCipherSuites; @@ -82,7 +82,7 @@ public FileSystemFactoryImpl(HdfsConfig config) { config.kerberosKdc, config.kerberosAuthorization, config.kerberosAutorenewal, - config.UseHdfsHostname, + config.useHdfsHostname, config.kerberosPrincipalPattern, config.hdfsTransferProtection, config.hdfsCipherSuites, @@ -111,7 +111,7 @@ public FileSystemFactoryImpl( this.kerberosKdc = kerberosKdc; this.kerberosAuthorization = kerberosAuthorization; this.kerberosAutorenewal = kerberosAutorenewal; - this.UseHdfsHostname = UseHdfsHostname; + this.useHdfsHostname = UseHdfsHostname; this.kerberosPrincipalPattern = kerberosPrincipalPattern; this.hdfsTransferProtection = hdfsTransferProtection; this.hdfsCipherSuites = hdfsCipherSuites; @@ -142,7 +142,7 @@ public FileSystem fileSystem(boolean initializeUGI) throws IOException { 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", UseHdfsHostname); + conf.set("dfs.client.use.datanode.hostname", useHdfsHostname); // server principal, the kerberos principle that the namenode is using conf.set("dfs.namenode.kerberos.principal.pattern", kerberosPrincipalPattern); diff --git a/src/main/java/com/teragrep/pth_06/config/HdfsConfig.java b/src/main/java/com/teragrep/pth_06/config/HdfsConfig.java index 25ef78b7..ce15b90f 100644 --- a/src/main/java/com/teragrep/pth_06/config/HdfsConfig.java +++ b/src/main/java/com/teragrep/pth_06/config/HdfsConfig.java @@ -55,7 +55,7 @@ public final class HdfsConfig { public final long includeRecordEpochAndAfter; // Represents the cutoff epoch in microseconds which dictates which record should not be fetched from HDFS based on the record timestamp. public final String hdfsPath; // Represents the working directory path in HDFS filesystem. public final String hdfsUri; // Represents the address of the HDFS server. - public final String UseHdfsHostname; + public final String useHdfsHostname; public final String hdfsTransferProtection; public final String hdfsCipherSuites; public final boolean useMockHdfsDatabase; // Represents the configuration parameter if mock database should be used or not. Used only for testing. @@ -90,7 +90,7 @@ public HdfsConfig(Map opts) { ); hdfsPath = getOrThrow(opts, "hdfs.hdfsPath"); hdfsUri = getOrThrow(opts, "hdfs.hdfsUri"); - UseHdfsHostname = getOrThrow(opts, "hdfs.UseHostName"); + useHdfsHostname = getOrThrow(opts, "hdfs.UseHostName"); hdfsTransferProtection = getOrThrow(opts, "hdfs.transferProtection"); hdfsCipherSuites = getOrThrow(opts, "hdfs.cipherSuites"); String useMockHdfsDatabaseString = opts.getOrDefault("hdfs.useMockHdfsDatabase", "false"); @@ -111,7 +111,7 @@ public HdfsConfig() { includeRecordEpochAndAfter = 0L; hdfsPath = ""; hdfsUri = ""; - UseHdfsHostname = ""; + useHdfsHostname = ""; hdfsTransferProtection = ""; hdfsCipherSuites = ""; useMockHdfsDatabase = false; diff --git a/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartition.java b/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartition.java index e9e7e509..487ed4ac 100644 --- a/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartition.java +++ b/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartition.java @@ -62,7 +62,7 @@ public class HdfsMicroBatchInputPartition implements InputPartition { public final long includeRecordEpochAndAfter; // Represents the cutoff epoch which dictates which files should not be fetched from HDFS based on their timestamps. public final String hdfsPath; // Represents the working directory path in HDFS filesystem. public final String hdfsUri; // Represents the address of the HDFS server. - public final String UseHdfsHostname; + public final String useHdfsHostname; public final String hdfsTransferProtection; public final String hdfsCipherSuites; public final boolean useMockHdfsDatabase; // Represents the configuration parameter if mock database should be used or not. Used only for testing. @@ -81,7 +81,7 @@ public HdfsMicroBatchInputPartition(HdfsConfig hdfsConfig, LinkedList Date: Wed, 16 Oct 2024 09:53:22 +0300 Subject: [PATCH 30/62] Cleaned up comments. --- .../scheduler/HdfsBatchSliceCollection.java | 34 ------------------- 1 file changed, 34 deletions(-) diff --git a/src/main/java/com/teragrep/pth_06/scheduler/HdfsBatchSliceCollection.java b/src/main/java/com/teragrep/pth_06/scheduler/HdfsBatchSliceCollection.java index 059a7ab1..ed334c8f 100644 --- a/src/main/java/com/teragrep/pth_06/scheduler/HdfsBatchSliceCollection.java +++ b/src/main/java/com/teragrep/pth_06/scheduler/HdfsBatchSliceCollection.java @@ -66,41 +66,7 @@ public HdfsBatchSliceCollection(HdfsQuery hq) { } public HdfsBatchSliceCollection processRange(Offset start, Offset end) { - /* - KAFKA: - KafkaOffset kafkaStartOffset = ((DatasourceOffset)start).getKafkaOffset(); - KafkaOffset kafkaEndOffset = ((DatasourceOffset)end).getKafkaOffset(); - KafkaBatchSliceCollection rv = generate(kafkaStartOffset, kafkaEndOffset); - LOGGER.debug("processRange(): arg start " + start + " arg end: " + end + " rv: " + rv ); - return rv; - - ARCHIVE: - LOGGER.debug("processRange(): args: start: " + start + " end: " + end); - this.clear(); // clear internal list - Result> - result = aq.processBetweenUnixEpochHours(((DatasourceOffset)start).getArchiveOffset().offset(), - ((DatasourceOffset)end).getArchiveOffset().offset()); - for (Record r : result) { - this.add( - new BatchSlice( - new ArchiveS3ObjectMetadata( - r.get(0, String.class), // id - r.get(6, String.class), // bucket - r.get(7, String.class), // path - r.get(1, String.class), // directory - r.get(2, String.class), // stream - r.get(3, String.class), // host - r.get(8, Long.class), // logtime - r.get(9, Long.class) // compressedSize - ) - ) - ); - } - return this; - * */ - // HDFS: - // If the slices are not distributed correctly, refactor the code to use the archive approach instead of kafka approach. LOGGER.debug("processRange(): args: start: " + start + " end: " + end); HdfsOffset hdfsStartOffset = ((DatasourceOffset) start).getHdfsOffset(); HdfsOffset hdfsEndOffset = ((DatasourceOffset) end).getHdfsOffset(); From 4bc1ecdd4fdcc10a3824ed9fdf400dea8fc8bbea Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Wed, 16 Oct 2024 15:23:26 +0300 Subject: [PATCH 31/62] Refactored HdfsDBClient constructor into secondary and primary constructor to allow providing custom FileSystem as input parameter. --- src/main/java/com/teragrep/pth_06/planner/HdfsDBClient.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/main/java/com/teragrep/pth_06/planner/HdfsDBClient.java b/src/main/java/com/teragrep/pth_06/planner/HdfsDBClient.java index 2a4fb7a7..7228a45b 100644 --- a/src/main/java/com/teragrep/pth_06/planner/HdfsDBClient.java +++ b/src/main/java/com/teragrep/pth_06/planner/HdfsDBClient.java @@ -66,10 +66,14 @@ public class HdfsDBClient { private final long ignoreBeforeEpoch; public HdfsDBClient(Config config, String topicsRegexString) throws IOException { + this(config, topicsRegexString, new FileSystemFactoryImpl(config.hdfsConfig).fileSystem(true)); + } + + public HdfsDBClient(Config config, String topicsRegexString, FileSystem fs) { this.topicsRegexString = topicsRegexString; this.ignoreBeforeEpoch = config.hdfsConfig.includeFileEpochAndAfter; // Defines the minimum time window / cutoff epoch for making sure that the files which metadata is fetched by the planner are not mistakenly deleted by cfe_39 pruning before tasker can process the records inside. path = config.hdfsConfig.hdfsPath; - fs = new FileSystemFactoryImpl(config.hdfsConfig).fileSystem(true); + this.fs = fs; } // this queries and pulls the distinct file metadata values to the partitionList according to the given query conditions (condition only applies to topic names in planner side). From 4c007bb9fde18bfa4a8c0de0515eba2b33da4f46 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Thu, 17 Oct 2024 09:49:31 +0300 Subject: [PATCH 32/62] Refactored PathFilter interface usage in HdfsDBClient by implementing a separate TopicFilter class. --- .../teragrep/pth_06/planner/HdfsDBClient.java | 12 +--- .../teragrep/pth_06/planner/TopicFilter.java | 63 +++++++++++++++++++ 2 files changed, 65 insertions(+), 10 deletions(-) create mode 100644 src/main/java/com/teragrep/pth_06/planner/TopicFilter.java diff --git a/src/main/java/com/teragrep/pth_06/planner/HdfsDBClient.java b/src/main/java/com/teragrep/pth_06/planner/HdfsDBClient.java index 7228a45b..056eaded 100644 --- a/src/main/java/com/teragrep/pth_06/planner/HdfsDBClient.java +++ b/src/main/java/com/teragrep/pth_06/planner/HdfsDBClient.java @@ -62,7 +62,7 @@ public class HdfsDBClient { private final Logger LOGGER = LoggerFactory.getLogger(HdfsDBClient.class); private final String path; private final FileSystem fs; - private final String topicsRegexString; + private final TopicFilter topicFilter; private final long ignoreBeforeEpoch; public HdfsDBClient(Config config, String topicsRegexString) throws IOException { @@ -70,7 +70,7 @@ public HdfsDBClient(Config config, String topicsRegexString) throws IOException } public HdfsDBClient(Config config, String topicsRegexString, FileSystem fs) { - this.topicsRegexString = topicsRegexString; + this.topicFilter = new TopicFilter(topicsRegexString); this.ignoreBeforeEpoch = config.hdfsConfig.includeFileEpochAndAfter; // Defines the minimum time window / cutoff epoch for making sure that the files which metadata is fetched by the planner are not mistakenly deleted by cfe_39 pruning before tasker can process the records inside. path = config.hdfsConfig.hdfsPath; this.fs = fs; @@ -114,12 +114,4 @@ public LinkedList pullToPartitionList() throws IOException { return rv; } - private final PathFilter topicFilter = new PathFilter() { - - @Override - public boolean accept(Path path) { - return path.getName().matches(topicsRegexString); // Catches the directory names. - } - }; - } diff --git a/src/main/java/com/teragrep/pth_06/planner/TopicFilter.java b/src/main/java/com/teragrep/pth_06/planner/TopicFilter.java new file mode 100644 index 00000000..7d727dcb --- /dev/null +++ b/src/main/java/com/teragrep/pth_06/planner/TopicFilter.java @@ -0,0 +1,63 @@ +/* + * Teragrep Archive Datasource (pth_06) + * 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.pth_06.planner; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathFilter; + +public class TopicFilter implements PathFilter { + + private final String topicsRegexString; + + public TopicFilter(String topicsRegexString) { + this.topicsRegexString = topicsRegexString; + } + + @Override + public boolean accept(Path path) { + return path.getName().matches(topicsRegexString); // Catches the directory names. + } +} From f34678b8b68f2f4ead3e747cc5ebee0b2d9ca829 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Thu, 17 Oct 2024 12:33:20 +0300 Subject: [PATCH 33/62] Renamed get() to record() in AvroRead interface. --- src/main/java/com/teragrep/pth_06/task/hdfs/AvroRead.java | 2 +- src/main/java/com/teragrep/pth_06/task/hdfs/AvroReadImpl.java | 2 +- .../teragrep/pth_06/task/hdfs/HdfsRecordConversionImpl.java | 2 +- .../java/com/teragrep/pth_06/task/hdfs/AvroReadImplTest.java | 4 ++-- 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/main/java/com/teragrep/pth_06/task/hdfs/AvroRead.java b/src/main/java/com/teragrep/pth_06/task/hdfs/AvroRead.java index 67d813f4..81eb36f6 100644 --- a/src/main/java/com/teragrep/pth_06/task/hdfs/AvroRead.java +++ b/src/main/java/com/teragrep/pth_06/task/hdfs/AvroRead.java @@ -55,7 +55,7 @@ public interface AvroRead { boolean next(); - SyslogRecord get(); + SyslogRecord record(); void close() throws IOException; diff --git a/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReadImpl.java b/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReadImpl.java index 53f62132..120e61ff 100644 --- a/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReadImpl.java +++ b/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReadImpl.java @@ -96,7 +96,7 @@ public boolean next() { } @Override - public SyslogRecord get() { + public SyslogRecord record() { return currentRecord; } diff --git a/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversionImpl.java b/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversionImpl.java index a7e3fccb..8956519d 100644 --- a/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversionImpl.java +++ b/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversionImpl.java @@ -98,7 +98,7 @@ public boolean next() { @Override public InternalRow get() { - SyslogRecord currentRecord = avroReadImpl.get(); + SyslogRecord currentRecord = avroReadImpl.record(); rowWriter.reset(); rowWriter.zeroOutNullBytes(); rowWriter.write(0, currentRecord.getTimestamp()); diff --git a/src/test/java/com/teragrep/pth_06/task/hdfs/AvroReadImplTest.java b/src/test/java/com/teragrep/pth_06/task/hdfs/AvroReadImplTest.java index 6a223a63..1b877caa 100644 --- a/src/test/java/com/teragrep/pth_06/task/hdfs/AvroReadImplTest.java +++ b/src/test/java/com/teragrep/pth_06/task/hdfs/AvroReadImplTest.java @@ -108,7 +108,7 @@ public void testReadAllRecords() { avroReadImpl1.open(); long rowNum = 0L; while (avroReadImpl1.next()) { - SyslogRecord syslogRecord = avroReadImpl1.get(); + SyslogRecord syslogRecord = avroReadImpl1.record(); Assertions.assertEquals(rowNum, syslogRecord.getOffset()); // Checks offsets of the consumed records which should range from 0 to 9. rowNum++; } @@ -124,7 +124,7 @@ public void testReadAllRecords() { AvroReadImpl avroReadImpl2 = new AvroReadImpl(fs, testConsumerTopic013); avroReadImpl2.open(); while (avroReadImpl2.next()) { - SyslogRecord syslogRecord = avroReadImpl2.get(); + SyslogRecord syslogRecord = avroReadImpl2.record(); Assertions.assertEquals(rowNum, syslogRecord.getOffset()); // Checks offsets of the consumed records which should range from 10 to 13. rowNum++; } From 6c91b0df996f403fee93ebcba7d2df7e98489282 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Thu, 17 Oct 2024 12:36:16 +0300 Subject: [PATCH 34/62] Renamed get() to row() in HdfsRecordConversion interface. --- .../pth_06/task/HdfsMicroBatchInputPartitionReader.java | 4 ++-- .../com/teragrep/pth_06/task/hdfs/HdfsRecordConversion.java | 2 +- .../teragrep/pth_06/task/hdfs/HdfsRecordConversionImpl.java | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReader.java b/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReader.java index c4f8e36d..475d05c9 100644 --- a/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReader.java +++ b/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReader.java @@ -132,7 +132,7 @@ public boolean next() throws IOException { } else { // time based inclusion, skip record and continue loop if the record is older than cutoffEpoch. - long rfc5424time = hdfsRecordConversionImpl.get().getLong(0); // timestamp as epochMicros + long rfc5424time = hdfsRecordConversionImpl.row().getLong(0); // timestamp as epochMicros if (rfc5424time < cutoffEpoch) { rv = false; } @@ -144,7 +144,7 @@ public boolean next() throws IOException { @Override public InternalRow get() { - InternalRow rv = hdfsRecordConversionImpl.get(); + InternalRow rv = hdfsRecordConversionImpl.row(); if (LOGGER.isDebugEnabled()) { LOGGER.debug("get(): <{}>", rv.getLong(7)); } diff --git a/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversion.java b/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversion.java index 0e1fb400..53bb4bee 100644 --- a/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversion.java +++ b/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversion.java @@ -57,7 +57,7 @@ public interface HdfsRecordConversion { boolean next(); - InternalRow get(); + InternalRow row(); boolean isStub(); diff --git a/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversionImpl.java b/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversionImpl.java index 8956519d..f6a24290 100644 --- a/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversionImpl.java +++ b/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversionImpl.java @@ -97,7 +97,7 @@ public boolean next() { } @Override - public InternalRow get() { + public InternalRow row() { SyslogRecord currentRecord = avroReadImpl.record(); rowWriter.reset(); rowWriter.zeroOutNullBytes(); From ce460834be4bfb6995f6b3e19b06b0b06373de2a Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Thu, 17 Oct 2024 16:32:46 +0300 Subject: [PATCH 35/62] Refactored primary constructor to only initialize all the encapsulated properties. --- .../pth_06/task/hdfs/HdfsRecordConversionImpl.java | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversionImpl.java b/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversionImpl.java index f6a24290..5a95b93d 100644 --- a/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversionImpl.java +++ b/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversionImpl.java @@ -75,10 +75,14 @@ public HdfsRecordConversionImpl(FileSystem fs, HdfsFileMetadata hdfsFileMetadata this(fs, hdfsFileMetadata, false); } - public HdfsRecordConversionImpl(FileSystem fs, HdfsFileMetadata hdfsFileMetadata, Boolean stub) { - this.avroReadImpl = new AvroReadImpl(fs, hdfsFileMetadata); + public HdfsRecordConversionImpl(FileSystem fs, HdfsFileMetadata hdfsFileMetadata, boolean stub) { + this(new AvroReadImpl(fs, hdfsFileMetadata), new UnsafeRowWriter(11), stub); + } + + public HdfsRecordConversionImpl(AvroReadImpl avroReadImpl, UnsafeRowWriter rowWriter, boolean stub) { + this.avroReadImpl = avroReadImpl; + this.rowWriter = rowWriter; this.stub = stub; - this.rowWriter = new UnsafeRowWriter(11); } @Override From 30a42202ef5a9b3436bf64c8fbc58460b497d6e8 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Mon, 21 Oct 2024 15:43:43 +0300 Subject: [PATCH 36/62] Refactoring AvroReadImpl.java, HdfsMicroBatchInputPartitionReader.java and HdfsRecordConversionImpl.java to make all objects immutable. Time based inclusion refactoring is WIP. --- .../HdfsMicroBatchInputPartitionReader.java | 52 ++----------- .../teragrep/pth_06/task/hdfs/AvroRead.java | 2 - .../pth_06/task/hdfs/AvroReadImpl.java | 36 ++------- .../task/hdfs/HdfsRecordConversion.java | 8 +- .../task/hdfs/HdfsRecordConversionImpl.java | 74 ++++++++++--------- .../pth_06/task/hdfs/AvroReadImplTest.java | 2 - ...dfsMicroBatchInputPartitionReaderTest.java | 6 +- 7 files changed, 55 insertions(+), 125 deletions(-) diff --git a/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReader.java b/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReader.java index 475d05c9..90f4792a 100644 --- a/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReader.java +++ b/src/main/java/com/teragrep/pth_06/task/HdfsMicroBatchInputPartitionReader.java @@ -60,11 +60,7 @@ public final class HdfsMicroBatchInputPartitionReader implements PartitionReader { final Logger LOGGER = LoggerFactory.getLogger(HdfsMicroBatchInputPartitionReader.class); - - private final LinkedList taskObjectList; - private final FileSystem fs; - private final long cutoffEpoch; - private HdfsRecordConversionImpl hdfsRecordConversionImpl; + private final HdfsRecordConversionImpl hdfsRecordConversionImpl; public HdfsMicroBatchInputPartitionReader( long cutoffEpoch, @@ -84,10 +80,7 @@ public HdfsMicroBatchInputPartitionReader( LinkedList taskObjectList ) throws IOException { - this.cutoffEpoch = cutoffEpoch; - this.taskObjectList = taskObjectList; - - fs = new FileSystemFactoryImpl( + FileSystem fs = new FileSystemFactoryImpl( kerberosAuthentication, hdfsUri, kerberosRealm, @@ -102,48 +95,18 @@ public HdfsMicroBatchInputPartitionReader( kerberosKeytabPath ).fileSystem(false); - this.hdfsRecordConversionImpl = new HdfsRecordConversionImpl(fs); // stub + this.hdfsRecordConversionImpl = new HdfsRecordConversionImpl(fs, taskObjectList, cutoffEpoch); } - // Read avro-file until it ends @Override public boolean next() throws IOException { - // true if data is available, false if not - boolean rv = false; - - while (!taskObjectList.isEmpty() && !rv) { - if (hdfsRecordConversionImpl.isStub()) { - hdfsRecordConversionImpl = new HdfsRecordConversionImpl(fs, taskObjectList.getFirst()); - hdfsRecordConversionImpl.open(); - } - - rv = hdfsRecordConversionImpl.next(); - - if (!rv) { - // object was consumed - hdfsRecordConversionImpl.close(); - - // remove consumed object - taskObjectList.removeFirst(); - if (!taskObjectList.isEmpty()) { - hdfsRecordConversionImpl = new HdfsRecordConversionImpl(fs, taskObjectList.getFirst()); - hdfsRecordConversionImpl.open(); - } - } - else { - // time based inclusion, skip record and continue loop if the record is older than cutoffEpoch. - long rfc5424time = hdfsRecordConversionImpl.row().getLong(0); // timestamp as epochMicros - if (rfc5424time < cutoffEpoch) { - rv = false; - } - } - } - LOGGER.debug("next rv: <{}>", rv); - return rv; + // True if data is available, false if not. + return hdfsRecordConversionImpl.next(); } @Override public InternalRow get() { + // Fetches the available data. InternalRow rv = hdfsRecordConversionImpl.row(); if (LOGGER.isDebugEnabled()) { LOGGER.debug("get(): <{}>", rv.getLong(7)); @@ -154,8 +117,5 @@ public InternalRow get() { @Override public void close() throws IOException { LOGGER.debug("HdfsMicroBatchInputPartitionReader.close"); - if (!hdfsRecordConversionImpl.isStub()) { - hdfsRecordConversionImpl.close(); - } } } diff --git a/src/main/java/com/teragrep/pth_06/task/hdfs/AvroRead.java b/src/main/java/com/teragrep/pth_06/task/hdfs/AvroRead.java index 81eb36f6..ac3f4390 100644 --- a/src/main/java/com/teragrep/pth_06/task/hdfs/AvroRead.java +++ b/src/main/java/com/teragrep/pth_06/task/hdfs/AvroRead.java @@ -51,8 +51,6 @@ public interface AvroRead { - void open() throws IOException; - boolean next(); SyslogRecord record(); diff --git a/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReadImpl.java b/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReadImpl.java index 120e61ff..9ce99fc1 100644 --- a/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReadImpl.java +++ b/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReadImpl.java @@ -49,7 +49,6 @@ import com.teragrep.pth_06.avro.SyslogRecord; 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.Path; import org.slf4j.Logger; @@ -62,42 +61,23 @@ public final class AvroReadImpl implements AvroRead { final Logger LOGGER = LoggerFactory.getLogger(AvroReadImpl.class); - private final FileSystem fs; - private final HdfsFileMetadata hdfsFileMetadata; - private DataFileStream reader; - private SyslogRecord currentRecord; + private final DataFileStream reader; - public AvroReadImpl(FileSystem fs, HdfsFileMetadata hdfsFileMetadata) { - this.fs = fs; - this.hdfsFileMetadata = hdfsFileMetadata; - } - - @Override - public void open() throws IOException { - Path hdfsreadpath = new Path(hdfsFileMetadata.hdfsFilePath); - if (LOGGER.isDebugEnabled()) { - LOGGER - .debug( - "Attempting to open file <{}> belonging to topic <{}>", hdfsreadpath.getName(), - hdfsFileMetadata.topicPartition.topic() - ); - } - FSDataInputStream inputStream = fs.open(hdfsreadpath); - reader = new DataFileStream<>(inputStream, new SpecificDatumReader<>(SyslogRecord.class)); + public AvroReadImpl(FileSystem fs, HdfsFileMetadata hdfsFileMetadata) throws IOException { + this.reader = new DataFileStream<>( + fs.open(new Path(hdfsFileMetadata.hdfsFilePath)), + new SpecificDatumReader<>(SyslogRecord.class) + ); } @Override public boolean next() { - boolean hasNext = reader.hasNext(); - if (hasNext) { - currentRecord = reader.next(); - } - return hasNext; + return reader.hasNext(); } @Override public SyslogRecord record() { - return currentRecord; + return reader.next(); } @Override diff --git a/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversion.java b/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversion.java index 53bb4bee..f14a2ab8 100644 --- a/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversion.java +++ b/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversion.java @@ -51,14 +51,8 @@ public interface HdfsRecordConversion { - void open() throws IOException; - - void close() throws IOException; - - boolean next(); + boolean next() throws IOException; InternalRow row(); - boolean isStub(); - } diff --git a/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversionImpl.java b/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversionImpl.java index 5a95b93d..f4496c35 100644 --- a/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversionImpl.java +++ b/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversionImpl.java @@ -48,7 +48,6 @@ import com.teragrep.pth_06.HdfsFileMetadata; import com.teragrep.pth_06.avro.SyslogRecord; import org.apache.hadoop.fs.FileSystem; -import org.apache.kafka.common.TopicPartition; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter; import org.apache.spark.unsafe.types.UTF8String; @@ -56,53 +55,54 @@ import org.slf4j.LoggerFactory; import java.io.IOException; +import java.util.LinkedList; // This class will read the records from avro-files fetched from HDFS with the help of AvroReadImpl and convert them to InternalRow used by pth_06. public final class HdfsRecordConversionImpl implements HdfsRecordConversion { final Logger LOGGER = LoggerFactory.getLogger(HdfsRecordConversionImpl.class); - private final boolean stub; - private final AvroReadImpl avroReadImpl; - private final UnsafeRowWriter rowWriter; + private final LinkedList reads; + private final LinkedList taskObjectList; + private final long cutoffEpoch; + private final FileSystem fs; - // Stub object - public HdfsRecordConversionImpl(FileSystem fs) { - this(fs, new HdfsFileMetadata(new TopicPartition("", 0), 0, "", 0), true); - } - - public HdfsRecordConversionImpl(FileSystem fs, HdfsFileMetadata hdfsFileMetadata) { - this(fs, hdfsFileMetadata, false); - } - - public HdfsRecordConversionImpl(FileSystem fs, HdfsFileMetadata hdfsFileMetadata, boolean stub) { - this(new AvroReadImpl(fs, hdfsFileMetadata), new UnsafeRowWriter(11), stub); - } - - public HdfsRecordConversionImpl(AvroReadImpl avroReadImpl, UnsafeRowWriter rowWriter, boolean stub) { - this.avroReadImpl = avroReadImpl; - this.rowWriter = rowWriter; - this.stub = stub; + public HdfsRecordConversionImpl(FileSystem fs, LinkedList taskObjectList, long cutoffEpoch) { + this.fs = fs; + this.taskObjectList = taskObjectList; + this.reads = new LinkedList<>(); + this.cutoffEpoch = cutoffEpoch; } @Override - public void open() throws IOException { - avroReadImpl.open(); - } - - @Override - public void close() throws IOException { - avroReadImpl.close(); - } + public boolean next() throws IOException { + // Load the taskObjects from taskObjectList to reads list. + if (reads.isEmpty() && !taskObjectList.isEmpty()) { + open(); + } - @Override - public boolean next() { - return avroReadImpl.next(); + boolean hasNext = false; + while (!hasNext && !reads.isEmpty()) { + hasNext = reads.getFirst().next(); + if (!hasNext) { + reads.getFirst().close(); + reads.removeFirst(); + } + else { + // Time based inclusion WIP + /*SyslogRecord record = reads.getFirst().record(); + if (record.getTimestamp() < cutoffEpoch) { + hasNext = false; + }*/ + } + } + return hasNext; } @Override public InternalRow row() { - SyslogRecord currentRecord = avroReadImpl.record(); + SyslogRecord currentRecord = reads.getFirst().record(); + UnsafeRowWriter rowWriter = new UnsafeRowWriter(11); rowWriter.reset(); rowWriter.zeroOutNullBytes(); rowWriter.write(0, currentRecord.getTimestamp()); @@ -117,9 +117,11 @@ public InternalRow row() { return rowWriter.getRow(); } - @Override - public boolean isStub() { - return stub; + private void open() throws IOException { + for (HdfsFileMetadata taskObject : taskObjectList) { + reads.add(new AvroReadImpl(fs, taskObject)); + } + taskObjectList.clear(); } } diff --git a/src/test/java/com/teragrep/pth_06/task/hdfs/AvroReadImplTest.java b/src/test/java/com/teragrep/pth_06/task/hdfs/AvroReadImplTest.java index 1b877caa..ba53a1ac 100644 --- a/src/test/java/com/teragrep/pth_06/task/hdfs/AvroReadImplTest.java +++ b/src/test/java/com/teragrep/pth_06/task/hdfs/AvroReadImplTest.java @@ -105,7 +105,6 @@ public void testReadAllRecords() { 0 ); AvroReadImpl avroReadImpl1 = new AvroReadImpl(fs, testConsumerTopic09); - avroReadImpl1.open(); long rowNum = 0L; while (avroReadImpl1.next()) { SyslogRecord syslogRecord = avroReadImpl1.record(); @@ -122,7 +121,6 @@ public void testReadAllRecords() { 0 ); AvroReadImpl avroReadImpl2 = new AvroReadImpl(fs, testConsumerTopic013); - avroReadImpl2.open(); while (avroReadImpl2.next()) { SyslogRecord syslogRecord = avroReadImpl2.record(); Assertions.assertEquals(rowNum, syslogRecord.getOffset()); // Checks offsets of the consumed records which should range from 10 to 13. diff --git a/src/test/java/com/teragrep/pth_06/task/hdfs/HdfsMicroBatchInputPartitionReaderTest.java b/src/test/java/com/teragrep/pth_06/task/hdfs/HdfsMicroBatchInputPartitionReaderTest.java index e86fa43b..a19e55b6 100644 --- a/src/test/java/com/teragrep/pth_06/task/hdfs/HdfsMicroBatchInputPartitionReaderTest.java +++ b/src/test/java/com/teragrep/pth_06/task/hdfs/HdfsMicroBatchInputPartitionReaderTest.java @@ -50,10 +50,7 @@ import com.teragrep.pth_06.task.HdfsMicroBatchInputPartitionReader; import org.apache.kafka.common.TopicPartition; import org.apache.spark.sql.catalyst.InternalRow; -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.junit.jupiter.api.*; import java.io.IOException; import java.util.LinkedList; @@ -198,6 +195,7 @@ public void testHdfsConsumer1FileAlt() { } + @Disabled(value = "This code is WIP after refactoring") @Test public void testCutoffEpoch() { assertDoesNotThrow(() -> { From 1a1063c1b106bbdc779c01214708af4f49816367 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Tue, 22 Oct 2024 10:50:16 +0300 Subject: [PATCH 37/62] Refactored ArchiveMicroStreamReader.java, DatasourceOffset.java and SerializedDatasourceOffset.java to reduce the amount of if-statements. --- .../pth_06/ArchiveMicroStreamReader.java | 127 ++++-------------- .../planner/offset/DatasourceOffset.java | 20 +-- .../offset/SerializedDatasourceOffset.java | 36 ----- 3 files changed, 37 insertions(+), 146 deletions(-) diff --git a/src/main/java/com/teragrep/pth_06/ArchiveMicroStreamReader.java b/src/main/java/com/teragrep/pth_06/ArchiveMicroStreamReader.java index 7151d2dc..a3795185 100644 --- a/src/main/java/com/teragrep/pth_06/ArchiveMicroStreamReader.java +++ b/src/main/java/com/teragrep/pth_06/ArchiveMicroStreamReader.java @@ -163,72 +163,30 @@ public final class ArchiveMicroStreamReader implements MicroBatchStream { */ @Override public Offset initialOffset() { - // After rebase is complete: Refactor the DatasourceOffset and SerializedDatasourceOffset if the 8x else-if statements are too much. - // archive only: subtract 3600s (1 hour) from earliest to return first row (start exclusive) DatasourceOffset rv; - if (this.config.isArchiveEnabled && !this.config.isKafkaEnabled && !this.config.isHdfsEnabled) { - // only archive - rv = new DatasourceOffset(new LongOffset(this.aq.getInitialOffset() - 3600L)); - } - else if (!this.config.isArchiveEnabled && this.config.isKafkaEnabled && !this.config.isHdfsEnabled) { - // only kafka - rv = new DatasourceOffset(new KafkaOffset(this.kq.getBeginningOffsets(null))); - } - else if (!this.config.isArchiveEnabled && !this.config.isKafkaEnabled && this.config.isHdfsEnabled) { - // only HDFS - rv = new DatasourceOffset(new HdfsOffset(this.hq.getBeginningOffsets().getOffsetMap())); - } - else if (this.config.isArchiveEnabled && this.config.isKafkaEnabled && !this.config.isHdfsEnabled) { - // kafka and archive - rv = new DatasourceOffset( - new LongOffset(this.aq.getInitialOffset() - 3600L), - new KafkaOffset(this.kq.getBeginningOffsets(null)) - ); - } - else if (this.config.isArchiveEnabled && !this.config.isKafkaEnabled && this.config.isHdfsEnabled) { - // archive and HDFS - rv = new DatasourceOffset( - new HdfsOffset(this.hq.getBeginningOffsets().getOffsetMap()), - new LongOffset(this.aq.getInitialOffset() - 3600L) - ); + HdfsOffset hdfsOffset = null; + LongOffset longOffset = null; + KafkaOffset kafkaOffset = null; + + if (this.config.isHdfsEnabled) { + hdfsOffset = new HdfsOffset(this.hq.getBeginningOffsets().getOffsetMap()); } - else if (!this.config.isArchiveEnabled && this.config.isKafkaEnabled && this.config.isHdfsEnabled) { - // Kafka and HDFS, check if any files are available from HDFS. - if (hdfsOffsets.size() > 0) { - rv = new DatasourceOffset( - new HdfsOffset(this.hq.getBeginningOffsets().getOffsetMap()), - new KafkaOffset(this.kq.getConsumerPositions(hdfsOffsets)) - ); - } - else { - rv = new DatasourceOffset( - new HdfsOffset(this.hq.getBeginningOffsets().getOffsetMap()), - new KafkaOffset(this.kq.getBeginningOffsets(null)) - ); - } + if (this.config.isArchiveEnabled) { + longOffset = new LongOffset(this.aq.getInitialOffset() - 3600L); } - else if (this.config.isArchiveEnabled && this.config.isKafkaEnabled && this.config.isHdfsEnabled) { - // all three, check if any files are available from HDFS. - if (hdfsOffsets.size() > 0) { - rv = new DatasourceOffset( - new HdfsOffset(this.hq.getBeginningOffsets().getOffsetMap()), - new LongOffset(this.aq.getInitialOffset() - 3600L), - new KafkaOffset(this.kq.getConsumerPositions(hdfsOffsets)) - ); + if (this.config.isKafkaEnabled) { + if (hdfsOffsets.size() > 0 && this.config.isHdfsEnabled) { + kafkaOffset = new KafkaOffset(this.kq.getConsumerPositions(hdfsOffsets)); } else { - rv = new DatasourceOffset( - new HdfsOffset(this.hq.getBeginningOffsets().getOffsetMap()), - new LongOffset(this.aq.getInitialOffset() - 3600L), - new KafkaOffset(this.kq.getBeginningOffsets(null)) - ); + kafkaOffset = new KafkaOffset(this.kq.getBeginningOffsets(null)); } } - else { - // none - throw new IllegalStateException("no datasources enabled, can't get initial offset"); + if (hdfsOffset == null && longOffset == null && kafkaOffset == null) { + throw new IllegalStateException("no datasources enabled, can't get latest offset"); } + rv = new DatasourceOffset(hdfsOffset, longOffset, kafkaOffset); LOGGER.debug("offset[initial]= {}", rv); return rv; } @@ -263,55 +221,24 @@ public void stop() { */ @Override public Offset latestOffset() { - // After rebase is complete: Refactor the DatasourceOffset and SerializedDatasourceOffset if the 8x else-if statements are too much. - DatasourceOffset rv; - if (this.config.isArchiveEnabled && !this.config.isKafkaEnabled && !this.config.isHdfsEnabled) { - // only archive - rv = new DatasourceOffset(new LongOffset(this.aq.incrementAndGetLatestOffset())); - } - else if (!this.config.isArchiveEnabled && this.config.isKafkaEnabled && !this.config.isHdfsEnabled) { - // only kafka - rv = new DatasourceOffset(new KafkaOffset(this.kq.getInitialEndOffsets())); - } - else if (!this.config.isArchiveEnabled && !this.config.isKafkaEnabled && this.config.isHdfsEnabled) { - // only hdfs - rv = new DatasourceOffset(new HdfsOffset(this.hq.incrementAndGetLatestOffset().getOffsetMap())); - } - else if (this.config.isArchiveEnabled && this.config.isKafkaEnabled && !this.config.isHdfsEnabled) { - // kafka and archive - rv = new DatasourceOffset( - new LongOffset(this.aq.incrementAndGetLatestOffset()), - new KafkaOffset(this.kq.getInitialEndOffsets()) - ); - } - else if (this.config.isArchiveEnabled && !this.config.isKafkaEnabled && this.config.isHdfsEnabled) { - // archive and hdfs - rv = new DatasourceOffset( - new HdfsOffset(this.hq.incrementAndGetLatestOffset().getOffsetMap()), - new LongOffset(this.aq.incrementAndGetLatestOffset()) - ); + HdfsOffset hdfsOffset = null; + LongOffset longOffset = null; + KafkaOffset kafkaOffset = null; + + if (this.config.isHdfsEnabled) { + hdfsOffset = new HdfsOffset(this.hq.incrementAndGetLatestOffset().getOffsetMap()); } - else if (!this.config.isArchiveEnabled && this.config.isKafkaEnabled && this.config.isHdfsEnabled) { - // Kafka and HDFS - rv = new DatasourceOffset( - new HdfsOffset(this.hq.incrementAndGetLatestOffset().getOffsetMap()), - new KafkaOffset(this.kq.getInitialEndOffsets()) - ); + if (this.config.isArchiveEnabled) { + longOffset = new LongOffset(this.aq.incrementAndGetLatestOffset()); } - else if (this.config.isArchiveEnabled && this.config.isKafkaEnabled && this.config.isHdfsEnabled) { - // all three - rv = new DatasourceOffset( - new HdfsOffset(this.hq.incrementAndGetLatestOffset().getOffsetMap()), - new LongOffset(this.aq.incrementAndGetLatestOffset()), - new KafkaOffset(this.kq.getInitialEndOffsets()) - ); + if (this.config.isKafkaEnabled) { + kafkaOffset = new KafkaOffset(this.kq.getInitialEndOffsets()); } - else { - // none + if (hdfsOffset == null && longOffset == null && kafkaOffset == null) { throw new IllegalStateException("no datasources enabled, can't get latest offset"); } - + rv = new DatasourceOffset(hdfsOffset, longOffset, kafkaOffset); LOGGER.debug("offset[latest]= {}", rv); return rv; } diff --git a/src/main/java/com/teragrep/pth_06/planner/offset/DatasourceOffset.java b/src/main/java/com/teragrep/pth_06/planner/offset/DatasourceOffset.java index c24dd9b7..9a90d3b2 100644 --- a/src/main/java/com/teragrep/pth_06/planner/offset/DatasourceOffset.java +++ b/src/main/java/com/teragrep/pth_06/planner/offset/DatasourceOffset.java @@ -63,32 +63,32 @@ public class DatasourceOffset extends Offset implements Serializable { SerializedDatasourceOffset serializedDatasourceOffset; - public DatasourceOffset(HdfsOffset hdfsOffset, LongOffset archiveOffset, KafkaOffset kafkaOffset) { - this.serializedDatasourceOffset = new SerializedDatasourceOffset(hdfsOffset, archiveOffset, kafkaOffset); - } - public DatasourceOffset(HdfsOffset hdfsOffset, KafkaOffset kafkaOffset) { - this.serializedDatasourceOffset = new SerializedDatasourceOffset(hdfsOffset, kafkaOffset); + this(hdfsOffset, null, kafkaOffset); } public DatasourceOffset(HdfsOffset hdfsOffset, LongOffset archiveOffset) { - this.serializedDatasourceOffset = new SerializedDatasourceOffset(hdfsOffset, archiveOffset); + this(hdfsOffset, archiveOffset, null); } public DatasourceOffset(LongOffset archiveOffset, KafkaOffset kafkaOffset) { - this.serializedDatasourceOffset = new SerializedDatasourceOffset(archiveOffset, kafkaOffset); + this(null, archiveOffset, kafkaOffset); } public DatasourceOffset(HdfsOffset hdfsOffset) { - this.serializedDatasourceOffset = new SerializedDatasourceOffset(hdfsOffset); + this(hdfsOffset, null, null); } public DatasourceOffset(LongOffset archiveOffset) { - this.serializedDatasourceOffset = new SerializedDatasourceOffset(archiveOffset); + this(null, archiveOffset, null); } public DatasourceOffset(KafkaOffset kafkaOffset) { - this.serializedDatasourceOffset = new SerializedDatasourceOffset(kafkaOffset); + this(null, null, kafkaOffset); + } + + public DatasourceOffset(HdfsOffset hdfsOffset, LongOffset archiveOffset, KafkaOffset kafkaOffset) { + this.serializedDatasourceOffset = new SerializedDatasourceOffset(hdfsOffset, archiveOffset, kafkaOffset); } public DatasourceOffset(String s) { diff --git a/src/main/java/com/teragrep/pth_06/planner/offset/SerializedDatasourceOffset.java b/src/main/java/com/teragrep/pth_06/planner/offset/SerializedDatasourceOffset.java index 68909e15..2d681dbd 100644 --- a/src/main/java/com/teragrep/pth_06/planner/offset/SerializedDatasourceOffset.java +++ b/src/main/java/com/teragrep/pth_06/planner/offset/SerializedDatasourceOffset.java @@ -71,42 +71,6 @@ public SerializedDatasourceOffset(HdfsOffset hdfsOffset, LongOffset archiveOffse this.kafkaOffset = kafkaOffset; } - public SerializedDatasourceOffset(LongOffset archiveOffset, KafkaOffset kafkaOffset) { - this.hdfsOffset = null; - this.archiveOffset = archiveOffset; - this.kafkaOffset = kafkaOffset; - } - - public SerializedDatasourceOffset(HdfsOffset hdfsOffset, LongOffset archiveOffset) { - this.hdfsOffset = hdfsOffset; - this.archiveOffset = archiveOffset; - this.kafkaOffset = null; - } - - public SerializedDatasourceOffset(HdfsOffset hdfsOffset, KafkaOffset kafkaOffset) { - this.hdfsOffset = hdfsOffset; - this.archiveOffset = null; - this.kafkaOffset = kafkaOffset; - } - - public SerializedDatasourceOffset(HdfsOffset hdfsOffset) { - this.hdfsOffset = hdfsOffset; - this.kafkaOffset = null; - this.archiveOffset = null; - } - - public SerializedDatasourceOffset(LongOffset archiveOffset) { - this.hdfsOffset = null; - this.kafkaOffset = null; - this.archiveOffset = archiveOffset; - } - - public SerializedDatasourceOffset(KafkaOffset kafkaOffset) { - this.hdfsOffset = null; - this.kafkaOffset = kafkaOffset; - this.archiveOffset = null; - } - @Override public String toString() { return "SerializedDatasourceOffset{" + "version=" + version + ", hdfsOffset" + hdfsOffset + ", archiveOffset=" From 72e3f775dca8963f03e13b48141343308b772ebb Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Tue, 22 Oct 2024 14:52:50 +0300 Subject: [PATCH 38/62] Refactored ArchiveMicroStreamReader.java, HdfsOffset.java and KafkaOffset.java to remove null usages. Fixed error in initialOffset() method. --- .../pth_06/ArchiveMicroStreamReader.java | 23 +++++++++++-------- .../pth_06/planner/offset/HdfsOffset.java | 16 ++++++++++++- .../pth_06/planner/offset/KafkaOffset.java | 15 ++++++++++++ 3 files changed, 44 insertions(+), 10 deletions(-) diff --git a/src/main/java/com/teragrep/pth_06/ArchiveMicroStreamReader.java b/src/main/java/com/teragrep/pth_06/ArchiveMicroStreamReader.java index a3795185..d2d56ed2 100644 --- a/src/main/java/com/teragrep/pth_06/ArchiveMicroStreamReader.java +++ b/src/main/java/com/teragrep/pth_06/ArchiveMicroStreamReader.java @@ -165,9 +165,9 @@ public final class ArchiveMicroStreamReader implements MicroBatchStream { public Offset initialOffset() { // archive only: subtract 3600s (1 hour) from earliest to return first row (start exclusive) DatasourceOffset rv; - HdfsOffset hdfsOffset = null; - LongOffset longOffset = null; - KafkaOffset kafkaOffset = null; + HdfsOffset hdfsOffset = new HdfsOffset(); // stub + LongOffset longOffset = null; // Refactor null usage + KafkaOffset kafkaOffset = new KafkaOffset(); // stub if (this.config.isHdfsEnabled) { hdfsOffset = new HdfsOffset(this.hq.getBeginningOffsets().getOffsetMap()); @@ -176,14 +176,19 @@ public Offset initialOffset() { longOffset = new LongOffset(this.aq.getInitialOffset() - 3600L); } if (this.config.isKafkaEnabled) { - if (hdfsOffsets.size() > 0 && this.config.isHdfsEnabled) { - kafkaOffset = new KafkaOffset(this.kq.getConsumerPositions(hdfsOffsets)); + if (this.config.isHdfsEnabled) { + if (hdfsOffsets.size() > 0) { + kafkaOffset = new KafkaOffset(this.kq.getConsumerPositions(hdfsOffsets)); + } + else { + kafkaOffset = new KafkaOffset(this.kq.getBeginningOffsets(null)); + } } else { kafkaOffset = new KafkaOffset(this.kq.getBeginningOffsets(null)); } } - if (hdfsOffset == null && longOffset == null && kafkaOffset == null) { + if (hdfsOffset.isStub() && longOffset == null && kafkaOffset.isStub()) { throw new IllegalStateException("no datasources enabled, can't get latest offset"); } rv = new DatasourceOffset(hdfsOffset, longOffset, kafkaOffset); @@ -222,9 +227,9 @@ public void stop() { @Override public Offset latestOffset() { DatasourceOffset rv; - HdfsOffset hdfsOffset = null; + HdfsOffset hdfsOffset = new HdfsOffset(); LongOffset longOffset = null; - KafkaOffset kafkaOffset = null; + KafkaOffset kafkaOffset = new KafkaOffset(); if (this.config.isHdfsEnabled) { hdfsOffset = new HdfsOffset(this.hq.incrementAndGetLatestOffset().getOffsetMap()); @@ -235,7 +240,7 @@ public Offset latestOffset() { if (this.config.isKafkaEnabled) { kafkaOffset = new KafkaOffset(this.kq.getInitialEndOffsets()); } - if (hdfsOffset == null && longOffset == null && kafkaOffset == null) { + if (hdfsOffset.isStub() && longOffset == null && kafkaOffset.isStub()) { throw new IllegalStateException("no datasources enabled, can't get latest offset"); } rv = new DatasourceOffset(hdfsOffset, longOffset, kafkaOffset); diff --git a/src/main/java/com/teragrep/pth_06/planner/offset/HdfsOffset.java b/src/main/java/com/teragrep/pth_06/planner/offset/HdfsOffset.java index 34a0eac2..ddffc092 100644 --- a/src/main/java/com/teragrep/pth_06/planner/offset/HdfsOffset.java +++ b/src/main/java/com/teragrep/pth_06/planner/offset/HdfsOffset.java @@ -60,19 +60,29 @@ public class HdfsOffset extends Offset implements Serializable { private final Map serializedHdfsOffset; + private final boolean stub; + + public HdfsOffset() { + this(new HashMap<>(), true); + } public HdfsOffset(Map offset) { + this(offset, false); + } + + public HdfsOffset(Map offset, boolean stub) { serializedHdfsOffset = new HashMap<>(offset.size()); for (Map.Entry entry : offset.entrySet()) { - serializedHdfsOffset.put(entry.getKey().toString(), entry.getValue()); // offset } + this.stub = stub; } public HdfsOffset(String s) { Gson gson = new Gson(); serializedHdfsOffset = gson.fromJson(s, new TypeToken>() { }.getType()); + stub = false; } public Map getOffsetMap() { @@ -91,6 +101,10 @@ public Map getOffsetMap() { return rv; } + public boolean isStub() { + return stub; + } + @Override public String json() { Gson gson = new Gson(); diff --git a/src/main/java/com/teragrep/pth_06/planner/offset/KafkaOffset.java b/src/main/java/com/teragrep/pth_06/planner/offset/KafkaOffset.java index bb87d60f..6865354d 100644 --- a/src/main/java/com/teragrep/pth_06/planner/offset/KafkaOffset.java +++ b/src/main/java/com/teragrep/pth_06/planner/offset/KafkaOffset.java @@ -67,18 +67,29 @@ public class KafkaOffset extends Offset implements Serializable { }.getType(); private final Map serializedKafkaOffset; + private final boolean stub; + + public KafkaOffset() { + this(new HashMap<>(), true); + } public KafkaOffset(Map offset) { + this(offset, false); + } + + public KafkaOffset(Map offset, boolean stub) { serializedKafkaOffset = new HashMap<>(offset.size()); for (Map.Entry entry : offset.entrySet()) { serializedKafkaOffset.put(entry.getKey().toString(), entry.getValue()); // offset } + this.stub = stub; } public KafkaOffset(String s) { Gson gson = new Gson(); serializedKafkaOffset = gson.fromJson(s, mapType); + stub = false; } public Map getOffsetMap() { @@ -97,6 +108,10 @@ public Map getOffsetMap() { return rv; } + public boolean isStub() { + return stub; + } + @Override public String json() { Gson gson = new Gson(); From 4c34f54add9f723397ca4010fb141da8ced347f5 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Wed, 23 Oct 2024 09:39:24 +0300 Subject: [PATCH 39/62] Replaced KafkaOffset and HdfsOffset null references with stubs. --- .../pth_06/planner/offset/DatasourceOffset.java | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/src/main/java/com/teragrep/pth_06/planner/offset/DatasourceOffset.java b/src/main/java/com/teragrep/pth_06/planner/offset/DatasourceOffset.java index 9a90d3b2..5b0db417 100644 --- a/src/main/java/com/teragrep/pth_06/planner/offset/DatasourceOffset.java +++ b/src/main/java/com/teragrep/pth_06/planner/offset/DatasourceOffset.java @@ -68,23 +68,23 @@ public DatasourceOffset(HdfsOffset hdfsOffset, KafkaOffset kafkaOffset) { } public DatasourceOffset(HdfsOffset hdfsOffset, LongOffset archiveOffset) { - this(hdfsOffset, archiveOffset, null); + this(hdfsOffset, archiveOffset, new KafkaOffset()); } public DatasourceOffset(LongOffset archiveOffset, KafkaOffset kafkaOffset) { - this(null, archiveOffset, kafkaOffset); + this(new HdfsOffset(), archiveOffset, kafkaOffset); } public DatasourceOffset(HdfsOffset hdfsOffset) { - this(hdfsOffset, null, null); + this(hdfsOffset, null, new KafkaOffset()); } public DatasourceOffset(LongOffset archiveOffset) { - this(null, archiveOffset, null); + this(new HdfsOffset(), archiveOffset, new KafkaOffset()); } public DatasourceOffset(KafkaOffset kafkaOffset) { - this(null, null, kafkaOffset); + this(new HdfsOffset(), null, kafkaOffset); } public DatasourceOffset(HdfsOffset hdfsOffset, LongOffset archiveOffset, KafkaOffset kafkaOffset) { @@ -107,8 +107,8 @@ public LongOffset getArchiveOffset() { public KafkaOffset getKafkaOffset() { KafkaOffset kafkaOffset = serializedDatasourceOffset.kafkaOffset; - if (kafkaOffset == null || kafkaOffset.getOffsetMap() == null) { - return null; + if (kafkaOffset.isStub() || kafkaOffset.getOffsetMap().isEmpty()) { + return kafkaOffset; } for (Map.Entry entry : kafkaOffset.getOffsetMap().entrySet()) { From 95ec8db53e97d3dff469f03b1bc7e4f66253177f Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Wed, 23 Oct 2024 11:10:50 +0300 Subject: [PATCH 40/62] Removed serializable map conversion logic away from HdfsOffset constructor. --- .../pth_06/ArchiveMicroStreamReader.java | 17 +++++++++++++++-- .../pth_06/planner/HdfsQueryProcessor.java | 18 +++++++++++++++--- .../pth_06/planner/offset/HdfsOffset.java | 11 ++++------- 3 files changed, 34 insertions(+), 12 deletions(-) diff --git a/src/main/java/com/teragrep/pth_06/ArchiveMicroStreamReader.java b/src/main/java/com/teragrep/pth_06/ArchiveMicroStreamReader.java index d2d56ed2..c4c9ffe5 100644 --- a/src/main/java/com/teragrep/pth_06/ArchiveMicroStreamReader.java +++ b/src/main/java/com/teragrep/pth_06/ArchiveMicroStreamReader.java @@ -57,14 +57,17 @@ import com.teragrep.pth_06.task.HdfsMicroBatchInputPartition; import com.teragrep.pth_06.task.TeragrepPartitionReaderFactory; import com.teragrep.pth_06.task.KafkaMicroBatchInputPartition; +import org.apache.kafka.common.TopicPartition; import org.apache.spark.sql.connector.read.InputPartition; import org.apache.spark.sql.connector.read.PartitionReaderFactory; import org.apache.spark.sql.connector.read.streaming.MicroBatchStream; import org.apache.spark.sql.connector.read.streaming.Offset; import org.apache.spark.sql.execution.streaming.LongOffset; +import java.util.HashMap; import java.util.LinkedList; import java.util.List; +import java.util.Map; // logger @@ -170,7 +173,12 @@ public Offset initialOffset() { KafkaOffset kafkaOffset = new KafkaOffset(); // stub if (this.config.isHdfsEnabled) { - hdfsOffset = new HdfsOffset(this.hq.getBeginningOffsets().getOffsetMap()); + Map offset = this.hq.getBeginningOffsets().getOffsetMap(); + Map serializedHdfsOffset = new HashMap<>(offset.size()); + for (Map.Entry entry : offset.entrySet()) { + serializedHdfsOffset.put(entry.getKey().toString(), entry.getValue()); // offset + } + hdfsOffset = new HdfsOffset(serializedHdfsOffset); } if (this.config.isArchiveEnabled) { longOffset = new LongOffset(this.aq.getInitialOffset() - 3600L); @@ -232,7 +240,12 @@ public Offset latestOffset() { KafkaOffset kafkaOffset = new KafkaOffset(); if (this.config.isHdfsEnabled) { - hdfsOffset = new HdfsOffset(this.hq.incrementAndGetLatestOffset().getOffsetMap()); + Map offset = this.hq.incrementAndGetLatestOffset().getOffsetMap(); + Map serializedHdfsOffset = new HashMap<>(offset.size()); + for (Map.Entry entry : offset.entrySet()) { + serializedHdfsOffset.put(entry.getKey().toString(), entry.getValue()); // offset + } + hdfsOffset = new HdfsOffset(serializedHdfsOffset); } if (this.config.isArchiveEnabled) { longOffset = new LongOffset(this.aq.incrementAndGetLatestOffset()); diff --git a/src/main/java/com/teragrep/pth_06/planner/HdfsQueryProcessor.java b/src/main/java/com/teragrep/pth_06/planner/HdfsQueryProcessor.java index b52242eb..6bdb319c 100644 --- a/src/main/java/com/teragrep/pth_06/planner/HdfsQueryProcessor.java +++ b/src/main/java/com/teragrep/pth_06/planner/HdfsQueryProcessor.java @@ -206,7 +206,11 @@ public HdfsOffset getBeginningOffsets() { } } } - return new HdfsOffset(startOffset); + Map serializedHdfsOffset = new HashMap<>(startOffset.size()); + for (Map.Entry entry : startOffset.entrySet()) { + serializedHdfsOffset.put(entry.getKey().toString(), entry.getValue()); // offset + } + return new HdfsOffset(serializedHdfsOffset); } // returns the end offsets for all available (aka. filtered) topic partitions. @@ -227,7 +231,11 @@ public HdfsOffset getInitialEndOffsets() { } } } - return new HdfsOffset(endOffset); + Map serializedHdfsOffset = new HashMap<>(endOffset.size()); + for (Map.Entry entry : endOffset.entrySet()) { + serializedHdfsOffset.put(entry.getKey().toString(), entry.getValue()); // offset + } + return new HdfsOffset(serializedHdfsOffset); } // Increments the latest offset values and returns that incremented offsets. Works by pulling data from the topicPartitionList until weight limit is reached. @@ -248,7 +256,11 @@ public HdfsOffset incrementAndGetLatestOffset() { batchSizeLimit.add(r.hdfsFileSize); } } - return new HdfsOffset(latestHdfsOffsetMap); + Map serializedHdfsOffset = new HashMap<>(latestHdfsOffsetMap.size()); + for (Map.Entry entry : latestHdfsOffsetMap.entrySet()) { + serializedHdfsOffset.put(entry.getKey().toString(), entry.getValue()); // offset + } + return new HdfsOffset(serializedHdfsOffset); } } diff --git a/src/main/java/com/teragrep/pth_06/planner/offset/HdfsOffset.java b/src/main/java/com/teragrep/pth_06/planner/offset/HdfsOffset.java index ddffc092..ebea489d 100644 --- a/src/main/java/com/teragrep/pth_06/planner/offset/HdfsOffset.java +++ b/src/main/java/com/teragrep/pth_06/planner/offset/HdfsOffset.java @@ -66,15 +66,12 @@ public HdfsOffset() { this(new HashMap<>(), true); } - public HdfsOffset(Map offset) { - this(offset, false); + public HdfsOffset(Map serializedHdfsOffset) { + this(serializedHdfsOffset, false); } - public HdfsOffset(Map offset, boolean stub) { - serializedHdfsOffset = new HashMap<>(offset.size()); - for (Map.Entry entry : offset.entrySet()) { - serializedHdfsOffset.put(entry.getKey().toString(), entry.getValue()); // offset - } + public HdfsOffset(Map serializedHdfsOffset, boolean stub) { + this.serializedHdfsOffset = serializedHdfsOffset; this.stub = stub; } From a5268f075dc594ad15343cb96f01a833ee5cbdfb Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Wed, 23 Oct 2024 13:24:04 +0300 Subject: [PATCH 41/62] Implemented stub to replace null usage for HdfsQuery. --- .../pth_06/ArchiveMicroStreamReader.java | 8 +++---- .../teragrep/pth_06/planner/HdfsQuery.java | 2 ++ .../pth_06/planner/HdfsQueryProcessor.java | 22 ++++++++++++++++--- .../pth_06/MockTeragrepDatasource.java | 2 +- 4 files changed, 26 insertions(+), 8 deletions(-) diff --git a/src/main/java/com/teragrep/pth_06/ArchiveMicroStreamReader.java b/src/main/java/com/teragrep/pth_06/ArchiveMicroStreamReader.java index c4c9ffe5..2a67d9dc 100644 --- a/src/main/java/com/teragrep/pth_06/ArchiveMicroStreamReader.java +++ b/src/main/java/com/teragrep/pth_06/ArchiveMicroStreamReader.java @@ -111,8 +111,8 @@ public final class ArchiveMicroStreamReader implements MicroBatchStream { hdfsOffsets = hq.hdfsOffsetMapToJSON(); } else { - this.hq = null; - hdfsOffsets = null; + this.hq = new HdfsQueryProcessor(); + hdfsOffsets = new JsonArray(); } if (config.isArchiveEnabled) { @@ -147,12 +147,12 @@ public final class ArchiveMicroStreamReader implements MicroBatchStream { this.aq = aq; // Uses its own hardcoded query string defined in MockTeragrepDatasource. this.kq = kq; // Skips using query string (and thus topic filtering) altogether. this.hq = hq; // Uses the query string from config for topic filtering. - if (this.hq != null && this.kq != null) { + if (!this.hq.isStub() && this.kq != null) { hdfsOffsets = this.hq.hdfsOffsetMapToJSON(); this.kq.seekToHdfsOffsets(hdfsOffsets); } else { - hdfsOffsets = null; + hdfsOffsets = new JsonArray(); } LOGGER.debug("@VisibleForTesting MicroBatchReader> initialized"); diff --git a/src/main/java/com/teragrep/pth_06/planner/HdfsQuery.java b/src/main/java/com/teragrep/pth_06/planner/HdfsQuery.java index 76188484..27105699 100644 --- a/src/main/java/com/teragrep/pth_06/planner/HdfsQuery.java +++ b/src/main/java/com/teragrep/pth_06/planner/HdfsQuery.java @@ -64,4 +64,6 @@ public interface HdfsQuery { HdfsOffset getInitialEndOffsets(); // Delete after testing incrementAndGetLatestOffset() functionality thoroughly. HdfsOffset incrementAndGetLatestOffset(); // replaces getInitialEndOffsets() + + boolean isStub(); } diff --git a/src/main/java/com/teragrep/pth_06/planner/HdfsQueryProcessor.java b/src/main/java/com/teragrep/pth_06/planner/HdfsQueryProcessor.java index 6bdb319c..6893265a 100644 --- a/src/main/java/com/teragrep/pth_06/planner/HdfsQueryProcessor.java +++ b/src/main/java/com/teragrep/pth_06/planner/HdfsQueryProcessor.java @@ -67,12 +67,22 @@ public class HdfsQueryProcessor implements HdfsQuery { private final Logger LOGGER = LoggerFactory.getLogger(HdfsQueryProcessor.class); private LinkedList topicPartitionList; private final HdfsDBClient hdfsDBClient; - private String topicsRegexString; private final Map hdfsOffsetMap; private final Map latestHdfsOffsetMap; private final long quantumLength; private final long numPartitions; private final long totalObjectCountLimit; + private final boolean stub; + + public HdfsQueryProcessor() { + totalObjectCountLimit = 0; + hdfsDBClient = null; // refactor null to stub + hdfsOffsetMap = new HashMap<>(); + latestHdfsOffsetMap = new HashMap<>(); + quantumLength = 0; + numPartitions = 0; + stub = true; + } public HdfsQueryProcessor(Config config) { // get configs from config object @@ -80,7 +90,7 @@ public HdfsQueryProcessor(Config config) { this.numPartitions = config.batchConfig.numPartitions; this.totalObjectCountLimit = config.batchConfig.totalObjectCountLimit; // Filter only topics using regex pattern - topicsRegexString = null; + String topicsRegexString = ""; if (config.query != null) { try { HdfsConditionWalker parser = new HdfsConditionWalker(); @@ -93,7 +103,7 @@ public HdfsQueryProcessor(Config config) { ); } } - if (topicsRegexString == null) { + if (topicsRegexString.isEmpty()) { topicsRegexString = "^.*$"; // all topics if none given } // Implement hdfs db client that fetches the metadata for the files that are stored in hdfs based on topic name (aka. directory containing the files for a specific topic in HDFS). @@ -124,6 +134,7 @@ public HdfsQueryProcessor(Config config) { } } latestHdfsOffsetMap = new HashMap<>(); + stub = false; LOGGER.debug("HdfsQueryProcessor.HdfsQueryProcessor>"); } @@ -239,6 +250,7 @@ public HdfsOffset getInitialEndOffsets() { } // Increments the latest offset values and returns that incremented offsets. Works by pulling data from the topicPartitionList until weight limit is reached. + @Override public HdfsOffset incrementAndGetLatestOffset() { if (this.latestHdfsOffsetMap.isEmpty()) { HdfsOffset beginningOffsets = getBeginningOffsets(); @@ -263,4 +275,8 @@ public HdfsOffset incrementAndGetLatestOffset() { return new HdfsOffset(serializedHdfsOffset); } + @Override + public boolean isStub() { + return stub; + } } diff --git a/src/test/java/com/teragrep/pth_06/MockTeragrepDatasource.java b/src/test/java/com/teragrep/pth_06/MockTeragrepDatasource.java index 60dccd23..8289bd1c 100644 --- a/src/test/java/com/teragrep/pth_06/MockTeragrepDatasource.java +++ b/src/test/java/com/teragrep/pth_06/MockTeragrepDatasource.java @@ -131,7 +131,7 @@ public MicroBatchStream toMicroBatchStream(String checkpointLocation) { hdfsQueryProcessor = new HdfsQueryProcessor(config); } else { - hdfsQueryProcessor = null; + hdfsQueryProcessor = new HdfsQueryProcessor(); } ArchiveQuery archiveQueryProcessor = new MockArchiveQueryProcessor( From e1379c5a46e787ed4529eaf2e8558e953f1318b3 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Wed, 23 Oct 2024 15:37:51 +0300 Subject: [PATCH 42/62] Implemented StubBatchSliceCollection.java to allow initialization of empty BatchSliceCollection. Replaced initialization of BatchSliceCollection as null with StubBatchSliceCollection. --- .../com/teragrep/pth_06/scheduler/Batch.java | 8 +-- .../scheduler/StubBatchSliceCollection.java | 60 +++++++++++++++++++ 2 files changed, 64 insertions(+), 4 deletions(-) create mode 100644 src/main/java/com/teragrep/pth_06/scheduler/StubBatchSliceCollection.java diff --git a/src/main/java/com/teragrep/pth_06/scheduler/Batch.java b/src/main/java/com/teragrep/pth_06/scheduler/Batch.java index 9f644d7d..daacf731 100644 --- a/src/main/java/com/teragrep/pth_06/scheduler/Batch.java +++ b/src/main/java/com/teragrep/pth_06/scheduler/Batch.java @@ -87,14 +87,14 @@ public Batch(Config config, HdfsQuery hq, ArchiveQuery aq, KafkaQuery kq) { public Batch processRange(Offset start, Offset end) { LOGGER.debug("processRange"); - BatchSliceCollection slice = null; + BatchSliceCollection slice = new StubBatchSliceCollection(); if (config.isHdfsEnabled) { slice = new HdfsBatchSliceCollection(this.hdfsQuery).processRange(start, end); } if (config.isArchiveEnabled) { - if (slice == null) { + if (slice.isEmpty()) { slice = new ArchiveBatchSliceCollection(this.archiveQuery).processRange(start, end); } else { @@ -103,14 +103,14 @@ public Batch processRange(Offset start, Offset end) { } if (config.isKafkaEnabled) { - if (slice == null) { + if (slice.isEmpty()) { slice = new KafkaBatchSliceCollection(this.kafkaQuery).processRange(start, end); } else { slice.addAll(new KafkaBatchSliceCollection(this.kafkaQuery).processRange(start, end)); } } - if (slice != null && !slice.isEmpty()) { + if (!slice.isEmpty()) { this.addSlice(slice); } diff --git a/src/main/java/com/teragrep/pth_06/scheduler/StubBatchSliceCollection.java b/src/main/java/com/teragrep/pth_06/scheduler/StubBatchSliceCollection.java new file mode 100644 index 00000000..6c72f23e --- /dev/null +++ b/src/main/java/com/teragrep/pth_06/scheduler/StubBatchSliceCollection.java @@ -0,0 +1,60 @@ +/* + * Teragrep Archive Datasource (pth_06) + * 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.pth_06.scheduler; + +import org.apache.spark.sql.connector.read.streaming.Offset; + +public final class StubBatchSliceCollection extends BatchSliceCollection { + + public StubBatchSliceCollection() { + super(); + } + + @Override + public BatchSliceCollection processRange(Offset start, Offset end) { + return this; + } +} From 3c01d45ca5e9d85bbda4616089cb51305ef0ea0a Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Thu, 24 Oct 2024 09:13:29 +0300 Subject: [PATCH 43/62] Refactored AvroRead and its implementation to restore hdfs time based inclusion functionality. --- .../teragrep/pth_06/task/hdfs/AvroRead.java | 2 ++ .../pth_06/task/hdfs/AvroReadImpl.java | 26 +++++++++++++++++-- .../task/hdfs/HdfsRecordConversionImpl.java | 6 +++-- .../pth_06/task/hdfs/AvroReadImplTest.java | 2 ++ ...dfsMicroBatchInputPartitionReaderTest.java | 1 - 5 files changed, 32 insertions(+), 5 deletions(-) diff --git a/src/main/java/com/teragrep/pth_06/task/hdfs/AvroRead.java b/src/main/java/com/teragrep/pth_06/task/hdfs/AvroRead.java index ac3f4390..3a865bb7 100644 --- a/src/main/java/com/teragrep/pth_06/task/hdfs/AvroRead.java +++ b/src/main/java/com/teragrep/pth_06/task/hdfs/AvroRead.java @@ -55,6 +55,8 @@ public interface AvroRead { SyslogRecord record(); + void clear(); + void close() throws IOException; } diff --git a/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReadImpl.java b/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReadImpl.java index 9ce99fc1..b4c6f883 100644 --- a/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReadImpl.java +++ b/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReadImpl.java @@ -55,6 +55,7 @@ import org.slf4j.LoggerFactory; import java.io.IOException; +import java.util.LinkedList; // This class will allow reading the contents of the avro-files that are using SyslogRecord schema from hdfs. public final class AvroReadImpl implements AvroRead { @@ -62,22 +63,43 @@ public final class AvroReadImpl implements AvroRead { final Logger LOGGER = LoggerFactory.getLogger(AvroReadImpl.class); private final DataFileStream reader; + private final LinkedList syslogRecordBuffer; public AvroReadImpl(FileSystem fs, HdfsFileMetadata hdfsFileMetadata) throws IOException { this.reader = new DataFileStream<>( fs.open(new Path(hdfsFileMetadata.hdfsFilePath)), new SpecificDatumReader<>(SyslogRecord.class) ); + syslogRecordBuffer = new LinkedList<>(); } @Override public boolean next() { - return reader.hasNext(); + boolean hasnext = reader.hasNext(); + if (hasnext) { + syslogRecordBuffer.add(reader.next()); + return true; + } + else { + return false; + } } @Override public SyslogRecord record() { - return reader.next(); + if (syslogRecordBuffer.size() == 1) { + return syslogRecordBuffer.getFirst(); + } + else { + throw new IllegalStateException( + "Invalid amount of records in the buffer, expected 1 got " + syslogRecordBuffer.size() + ); + } + } + + @Override + public void clear() { + syslogRecordBuffer.clear(); } @Override diff --git a/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversionImpl.java b/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversionImpl.java index f4496c35..9fb286fe 100644 --- a/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversionImpl.java +++ b/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversionImpl.java @@ -90,10 +90,11 @@ public boolean next() throws IOException { } else { // Time based inclusion WIP - /*SyslogRecord record = reads.getFirst().record(); + SyslogRecord record = reads.getFirst().record(); if (record.getTimestamp() < cutoffEpoch) { + reads.getFirst().clear(); hasNext = false; - }*/ + } } } return hasNext; @@ -102,6 +103,7 @@ public boolean next() throws IOException { @Override public InternalRow row() { SyslogRecord currentRecord = reads.getFirst().record(); + reads.getFirst().clear(); UnsafeRowWriter rowWriter = new UnsafeRowWriter(11); rowWriter.reset(); rowWriter.zeroOutNullBytes(); diff --git a/src/test/java/com/teragrep/pth_06/task/hdfs/AvroReadImplTest.java b/src/test/java/com/teragrep/pth_06/task/hdfs/AvroReadImplTest.java index ba53a1ac..b885ac3e 100644 --- a/src/test/java/com/teragrep/pth_06/task/hdfs/AvroReadImplTest.java +++ b/src/test/java/com/teragrep/pth_06/task/hdfs/AvroReadImplTest.java @@ -108,6 +108,7 @@ public void testReadAllRecords() { long rowNum = 0L; while (avroReadImpl1.next()) { SyslogRecord syslogRecord = avroReadImpl1.record(); + avroReadImpl1.clear(); Assertions.assertEquals(rowNum, syslogRecord.getOffset()); // Checks offsets of the consumed records which should range from 0 to 9. rowNum++; } @@ -123,6 +124,7 @@ public void testReadAllRecords() { AvroReadImpl avroReadImpl2 = new AvroReadImpl(fs, testConsumerTopic013); while (avroReadImpl2.next()) { SyslogRecord syslogRecord = avroReadImpl2.record(); + avroReadImpl2.clear(); Assertions.assertEquals(rowNum, syslogRecord.getOffset()); // Checks offsets of the consumed records which should range from 10 to 13. rowNum++; } diff --git a/src/test/java/com/teragrep/pth_06/task/hdfs/HdfsMicroBatchInputPartitionReaderTest.java b/src/test/java/com/teragrep/pth_06/task/hdfs/HdfsMicroBatchInputPartitionReaderTest.java index a19e55b6..e2afd265 100644 --- a/src/test/java/com/teragrep/pth_06/task/hdfs/HdfsMicroBatchInputPartitionReaderTest.java +++ b/src/test/java/com/teragrep/pth_06/task/hdfs/HdfsMicroBatchInputPartitionReaderTest.java @@ -195,7 +195,6 @@ public void testHdfsConsumer1FileAlt() { } - @Disabled(value = "This code is WIP after refactoring") @Test public void testCutoffEpoch() { assertDoesNotThrow(() -> { From b38922398af12a6afc4d0e73b9aa5dcdfdab971d Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Thu, 24 Oct 2024 12:24:09 +0300 Subject: [PATCH 44/62] Combined 3 for-loops into one. --- .../pth_06/ArchiveMicroStreamReader.java | 51 +++++++------------ 1 file changed, 18 insertions(+), 33 deletions(-) diff --git a/src/main/java/com/teragrep/pth_06/ArchiveMicroStreamReader.java b/src/main/java/com/teragrep/pth_06/ArchiveMicroStreamReader.java index 2a67d9dc..40aa9953 100644 --- a/src/main/java/com/teragrep/pth_06/ArchiveMicroStreamReader.java +++ b/src/main/java/com/teragrep/pth_06/ArchiveMicroStreamReader.java @@ -275,15 +275,31 @@ public InputPartition[] planInputPartitions(Offset start, Offset end) { Batch currentBatch = new Batch(config, hq, aq, kq).processRange(start, end); for (LinkedList taskObjectList : currentBatch) { - // archive tasks LinkedList archiveTaskList = new LinkedList<>(); + // HDFS tasks + LinkedList hdfsTaskList = new LinkedList<>(); for (BatchSlice batchSlice : taskObjectList) { if (batchSlice.type.equals(BatchSlice.Type.ARCHIVE)) { archiveTaskList.add(batchSlice.archiveS3ObjectMetadata); } + if (batchSlice.type.equals(BatchSlice.Type.HDFS)) { + hdfsTaskList.add(batchSlice.hdfsFileMetadata); + } + if (batchSlice.type.equals(BatchSlice.Type.KAFKA)) { + inputPartitions + .add( + new KafkaMicroBatchInputPartition( + config.kafkaConfig.executorOpts, + batchSlice.kafkaTopicPartitionOffsetMetadata.topicPartition, + batchSlice.kafkaTopicPartitionOffsetMetadata.startOffset, + batchSlice.kafkaTopicPartitionOffsetMetadata.endOffset, + config.kafkaConfig.executorConfig, + config.kafkaConfig.skipNonRFC5424Records + ) + ); + } } - if (!archiveTaskList.isEmpty()) { inputPartitions .add( @@ -300,40 +316,9 @@ public InputPartition[] planInputPartitions(Offset start, Offset end) { ) ); } - - // HDFS tasks - LinkedList hdfsTaskList = new LinkedList<>(); - for (BatchSlice batchSlice : taskObjectList) { - if (batchSlice.type.equals(BatchSlice.Type.HDFS)) { - hdfsTaskList.add(batchSlice.hdfsFileMetadata); - } - } - if (!hdfsTaskList.isEmpty()) { - /* BatchSliceType.HDFS contains the metadata for the HDFS files that contain the records that are being queried. Available topics in HDFS are already filtered based on the spark query conditions. - The records that are inside the files are fetched and processed in the tasker. Tasker does rest of the filtering based on the given query conditions, for example the cutoff epoch handling between the records that are fetched from S3 and HDFS. - The Spark planner/scheduler is only single-threaded while tasker is multithreaded. Planner is not suitable for fetching and processing all the records, it should be done in tasker which will handle the processing in multithreaded environment based on batch slices. - The slice creation has been changed to be incremental, which means that not all avro-files are fetched from hdfs at once. Instead, the files are distributed over several batches based on file size.*/ - inputPartitions.add(new HdfsMicroBatchInputPartition(config.hdfsConfig, hdfsTaskList)); } - - // kafka tasks - for (BatchSlice batchSlice : taskObjectList) { - if (batchSlice.type.equals(BatchSlice.Type.KAFKA)) { - inputPartitions - .add( - new KafkaMicroBatchInputPartition( - config.kafkaConfig.executorOpts, - batchSlice.kafkaTopicPartitionOffsetMetadata.topicPartition, - batchSlice.kafkaTopicPartitionOffsetMetadata.startOffset, - batchSlice.kafkaTopicPartitionOffsetMetadata.endOffset, - config.kafkaConfig.executorConfig, - config.kafkaConfig.skipNonRFC5424Records - ) - ); - } - } } return inputPartitions.toArray(new InputPartition[0]); From efe3f8bac1e5bf8edb5472b2a3784bf4d03e772a Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Fri, 25 Oct 2024 13:49:57 +0300 Subject: [PATCH 45/62] Fixed unintentional capital letter usage in config key. --- src/main/java/com/teragrep/pth_06/config/HdfsConfig.java | 2 +- src/test/java/com/teragrep/pth_06/InstantiationTest.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/java/com/teragrep/pth_06/config/HdfsConfig.java b/src/main/java/com/teragrep/pth_06/config/HdfsConfig.java index ce15b90f..4b7bc402 100644 --- a/src/main/java/com/teragrep/pth_06/config/HdfsConfig.java +++ b/src/main/java/com/teragrep/pth_06/config/HdfsConfig.java @@ -90,7 +90,7 @@ public HdfsConfig(Map opts) { ); hdfsPath = getOrThrow(opts, "hdfs.hdfsPath"); hdfsUri = getOrThrow(opts, "hdfs.hdfsUri"); - useHdfsHostname = getOrThrow(opts, "hdfs.UseHostName"); + useHdfsHostname = getOrThrow(opts, "hdfs.useHostName"); hdfsTransferProtection = getOrThrow(opts, "hdfs.transferProtection"); hdfsCipherSuites = getOrThrow(opts, "hdfs.cipherSuites"); String useMockHdfsDatabaseString = opts.getOrDefault("hdfs.useMockHdfsDatabase", "false"); diff --git a/src/test/java/com/teragrep/pth_06/InstantiationTest.java b/src/test/java/com/teragrep/pth_06/InstantiationTest.java index 9fb7104e..9765cd81 100644 --- a/src/test/java/com/teragrep/pth_06/InstantiationTest.java +++ b/src/test/java/com/teragrep/pth_06/InstantiationTest.java @@ -160,7 +160,7 @@ public void fullScanTest() throws StreamingQueryException, TimeoutException { .option("hdfs.enabled", "true") .option("hdfs.hdfsPath", hdfsPath) .option("hdfs.hdfsUri", hdfsUri) - .option("hdfs.UseHostName", "false") + .option("hdfs.useHostName", "false") .option("hdfs.transferProtection", "test") .option("hdfs.cipherSuites", "test") .option("hdfs.useMockHdfsDatabase", "true") From a83ba845b6a88972473e5512c4319dba8319a5df Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Fri, 25 Oct 2024 13:53:49 +0300 Subject: [PATCH 46/62] Moved TopicFilter initialization to secondary constructor. --- src/main/java/com/teragrep/pth_06/planner/HdfsDBClient.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/main/java/com/teragrep/pth_06/planner/HdfsDBClient.java b/src/main/java/com/teragrep/pth_06/planner/HdfsDBClient.java index 056eaded..136f0877 100644 --- a/src/main/java/com/teragrep/pth_06/planner/HdfsDBClient.java +++ b/src/main/java/com/teragrep/pth_06/planner/HdfsDBClient.java @@ -66,11 +66,11 @@ public class HdfsDBClient { private final long ignoreBeforeEpoch; public HdfsDBClient(Config config, String topicsRegexString) throws IOException { - this(config, topicsRegexString, new FileSystemFactoryImpl(config.hdfsConfig).fileSystem(true)); + this(config, new TopicFilter(topicsRegexString), new FileSystemFactoryImpl(config.hdfsConfig).fileSystem(true)); } - public HdfsDBClient(Config config, String topicsRegexString, FileSystem fs) { - this.topicFilter = new TopicFilter(topicsRegexString); + public HdfsDBClient(Config config, TopicFilter topicFilter, FileSystem fs) { + this.topicFilter = topicFilter; this.ignoreBeforeEpoch = config.hdfsConfig.includeFileEpochAndAfter; // Defines the minimum time window / cutoff epoch for making sure that the files which metadata is fetched by the planner are not mistakenly deleted by cfe_39 pruning before tasker can process the records inside. path = config.hdfsConfig.hdfsPath; this.fs = fs; From 05304ebc2d174c193229614aca6272950a15884b Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Fri, 25 Oct 2024 14:18:06 +0300 Subject: [PATCH 47/62] Initialize topicsRegexString with all topics instead of empty string. --- .../java/com/teragrep/pth_06/planner/HdfsQueryProcessor.java | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/src/main/java/com/teragrep/pth_06/planner/HdfsQueryProcessor.java b/src/main/java/com/teragrep/pth_06/planner/HdfsQueryProcessor.java index 6893265a..0eb04580 100644 --- a/src/main/java/com/teragrep/pth_06/planner/HdfsQueryProcessor.java +++ b/src/main/java/com/teragrep/pth_06/planner/HdfsQueryProcessor.java @@ -90,7 +90,7 @@ public HdfsQueryProcessor(Config config) { this.numPartitions = config.batchConfig.numPartitions; this.totalObjectCountLimit = config.batchConfig.totalObjectCountLimit; // Filter only topics using regex pattern - String topicsRegexString = ""; + String topicsRegexString = "^.*$"; // all topics if none given if (config.query != null) { try { HdfsConditionWalker parser = new HdfsConditionWalker(); @@ -103,9 +103,6 @@ public HdfsQueryProcessor(Config config) { ); } } - if (topicsRegexString.isEmpty()) { - topicsRegexString = "^.*$"; // all topics if none given - } // Implement hdfs db client that fetches the metadata for the files that are stored in hdfs based on topic name (aka. directory containing the files for a specific topic in HDFS). // Remember to implement Kerberized HDFS access for prod. Tests are done using normal access on mini cluster. try { From 968ab8518ac681c3703b3f1f41f7d95b91448464 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Fri, 25 Oct 2024 15:05:18 +0300 Subject: [PATCH 48/62] Implemented clearing of syslogRecordBuffer list as part of next() instead of separate clear() method. Removed clear() from AvroRead interface. --- src/main/java/com/teragrep/pth_06/task/hdfs/AvroRead.java | 2 -- .../java/com/teragrep/pth_06/task/hdfs/AvroReadImpl.java | 6 +----- .../teragrep/pth_06/task/hdfs/HdfsRecordConversionImpl.java | 2 -- .../com/teragrep/pth_06/task/hdfs/AvroReadImplTest.java | 2 -- 4 files changed, 1 insertion(+), 11 deletions(-) diff --git a/src/main/java/com/teragrep/pth_06/task/hdfs/AvroRead.java b/src/main/java/com/teragrep/pth_06/task/hdfs/AvroRead.java index 3a865bb7..ac3f4390 100644 --- a/src/main/java/com/teragrep/pth_06/task/hdfs/AvroRead.java +++ b/src/main/java/com/teragrep/pth_06/task/hdfs/AvroRead.java @@ -55,8 +55,6 @@ public interface AvroRead { SyslogRecord record(); - void clear(); - void close() throws IOException; } diff --git a/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReadImpl.java b/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReadImpl.java index b4c6f883..99d40b2f 100644 --- a/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReadImpl.java +++ b/src/main/java/com/teragrep/pth_06/task/hdfs/AvroReadImpl.java @@ -77,6 +77,7 @@ public AvroReadImpl(FileSystem fs, HdfsFileMetadata hdfsFileMetadata) throws IOE public boolean next() { boolean hasnext = reader.hasNext(); if (hasnext) { + syslogRecordBuffer.clear(); syslogRecordBuffer.add(reader.next()); return true; } @@ -97,11 +98,6 @@ public SyslogRecord record() { } } - @Override - public void clear() { - syslogRecordBuffer.clear(); - } - @Override public void close() throws IOException { reader.close(); diff --git a/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversionImpl.java b/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversionImpl.java index 9fb286fe..62bfb059 100644 --- a/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversionImpl.java +++ b/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversionImpl.java @@ -92,7 +92,6 @@ public boolean next() throws IOException { // Time based inclusion WIP SyslogRecord record = reads.getFirst().record(); if (record.getTimestamp() < cutoffEpoch) { - reads.getFirst().clear(); hasNext = false; } } @@ -103,7 +102,6 @@ public boolean next() throws IOException { @Override public InternalRow row() { SyslogRecord currentRecord = reads.getFirst().record(); - reads.getFirst().clear(); UnsafeRowWriter rowWriter = new UnsafeRowWriter(11); rowWriter.reset(); rowWriter.zeroOutNullBytes(); diff --git a/src/test/java/com/teragrep/pth_06/task/hdfs/AvroReadImplTest.java b/src/test/java/com/teragrep/pth_06/task/hdfs/AvroReadImplTest.java index b885ac3e..ba53a1ac 100644 --- a/src/test/java/com/teragrep/pth_06/task/hdfs/AvroReadImplTest.java +++ b/src/test/java/com/teragrep/pth_06/task/hdfs/AvroReadImplTest.java @@ -108,7 +108,6 @@ public void testReadAllRecords() { long rowNum = 0L; while (avroReadImpl1.next()) { SyslogRecord syslogRecord = avroReadImpl1.record(); - avroReadImpl1.clear(); Assertions.assertEquals(rowNum, syslogRecord.getOffset()); // Checks offsets of the consumed records which should range from 0 to 9. rowNum++; } @@ -124,7 +123,6 @@ public void testReadAllRecords() { AvroReadImpl avroReadImpl2 = new AvroReadImpl(fs, testConsumerTopic013); while (avroReadImpl2.next()) { SyslogRecord syslogRecord = avroReadImpl2.record(); - avroReadImpl2.clear(); Assertions.assertEquals(rowNum, syslogRecord.getOffset()); // Checks offsets of the consumed records which should range from 10 to 13. rowNum++; } From 56b3ef22ee113882e3ef0290fc336e779703be74 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Fri, 25 Oct 2024 15:29:15 +0300 Subject: [PATCH 49/62] Refactored StubBatchSliceCollection to throw exception when it is processed. --- .../com/teragrep/pth_06/scheduler/StubBatchSliceCollection.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/com/teragrep/pth_06/scheduler/StubBatchSliceCollection.java b/src/main/java/com/teragrep/pth_06/scheduler/StubBatchSliceCollection.java index 6c72f23e..2afa3d53 100644 --- a/src/main/java/com/teragrep/pth_06/scheduler/StubBatchSliceCollection.java +++ b/src/main/java/com/teragrep/pth_06/scheduler/StubBatchSliceCollection.java @@ -55,6 +55,6 @@ public StubBatchSliceCollection() { @Override public BatchSliceCollection processRange(Offset start, Offset end) { - return this; + throw new UnsupportedOperationException("StubBatchSliceCollection should not be processed."); } } From d0cef90eb8e3d6122b4199fc596d59f01669ee89 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Mon, 28 Oct 2024 10:44:05 +0200 Subject: [PATCH 50/62] Refactored HdfsOffset constructors and added hdfs offset serde test. --- .../pth_06/planner/offset/HdfsOffset.java | 12 ++++------ .../pth_06/planner/DatasourceOffsetTest.java | 24 +++++++++++++++++++ 2 files changed, 29 insertions(+), 7 deletions(-) diff --git a/src/main/java/com/teragrep/pth_06/planner/offset/HdfsOffset.java b/src/main/java/com/teragrep/pth_06/planner/offset/HdfsOffset.java index ebea489d..e9442d53 100644 --- a/src/main/java/com/teragrep/pth_06/planner/offset/HdfsOffset.java +++ b/src/main/java/com/teragrep/pth_06/planner/offset/HdfsOffset.java @@ -66,6 +66,11 @@ public HdfsOffset() { this(new HashMap<>(), true); } + public HdfsOffset(String s) { + this(new Gson().fromJson(s, new TypeToken>() { + }.getType()), false); + } + public HdfsOffset(Map serializedHdfsOffset) { this(serializedHdfsOffset, false); } @@ -75,13 +80,6 @@ public HdfsOffset(Map serializedHdfsOffset, boolean stub) { this.stub = stub; } - public HdfsOffset(String s) { - Gson gson = new Gson(); - serializedHdfsOffset = gson.fromJson(s, new TypeToken>() { - }.getType()); - stub = false; - } - public Map getOffsetMap() { Map rv = new HashMap<>(serializedHdfsOffset.size()); diff --git a/src/test/java/com/teragrep/pth_06/planner/DatasourceOffsetTest.java b/src/test/java/com/teragrep/pth_06/planner/DatasourceOffsetTest.java index 0f329712..45a9c978 100644 --- a/src/test/java/com/teragrep/pth_06/planner/DatasourceOffsetTest.java +++ b/src/test/java/com/teragrep/pth_06/planner/DatasourceOffsetTest.java @@ -46,6 +46,7 @@ package com.teragrep.pth_06.planner; import com.teragrep.pth_06.planner.offset.DatasourceOffset; +import com.teragrep.pth_06.planner.offset.HdfsOffset; import com.teragrep.pth_06.planner.offset.KafkaOffset; import org.apache.kafka.common.TopicPartition; import org.apache.spark.sql.execution.streaming.LongOffset; @@ -90,4 +91,27 @@ public void kafkaOffsetSerdeTest() { Assertions.assertEquals(9999L, offset); } } + + @Test + public void HdfsOffsetSerdeTest() { + Map topicPartitionLongMap = new HashMap<>(); + topicPartitionLongMap.put(new TopicPartition("test", 777), 9999L); + Map serializedHdfsOffset = new HashMap<>(topicPartitionLongMap.size()); + for (Map.Entry entry : topicPartitionLongMap.entrySet()) { + serializedHdfsOffset.put(entry.getKey().toString(), entry.getValue()); + } + HdfsOffset hdfsOffset = new HdfsOffset(serializedHdfsOffset); + + String ser = hdfsOffset.json(); + HdfsOffset deser = new HdfsOffset(ser); + + for (Map.Entry entry : deser.getOffsetMap().entrySet()) { + TopicPartition topicPartition = entry.getKey(); + long offset = entry.getValue(); + + Assertions.assertEquals("test", topicPartition.topic()); + Assertions.assertEquals(777, topicPartition.partition()); + Assertions.assertEquals(9999L, offset); + } + } } From 26e398776d92da24c1e1ac1bfaf2a03f807a7680 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Mon, 28 Oct 2024 13:00:52 +0200 Subject: [PATCH 51/62] Removed serializable map conversion logic away from KafkaOffset constructor. Refactored KafkaOffset constructors. --- .../pth_06/ArchiveMicroStreamReader.java | 26 ++++++++++++++++--- .../pth_06/planner/offset/KafkaOffset.java | 25 ++++++------------ .../pth_06/planner/DatasourceOffsetTest.java | 12 +++++++-- 3 files changed, 40 insertions(+), 23 deletions(-) diff --git a/src/main/java/com/teragrep/pth_06/ArchiveMicroStreamReader.java b/src/main/java/com/teragrep/pth_06/ArchiveMicroStreamReader.java index 40aa9953..75c23940 100644 --- a/src/main/java/com/teragrep/pth_06/ArchiveMicroStreamReader.java +++ b/src/main/java/com/teragrep/pth_06/ArchiveMicroStreamReader.java @@ -186,14 +186,28 @@ public Offset initialOffset() { if (this.config.isKafkaEnabled) { if (this.config.isHdfsEnabled) { if (hdfsOffsets.size() > 0) { - kafkaOffset = new KafkaOffset(this.kq.getConsumerPositions(hdfsOffsets)); + Map serializedKafkaOffset = new HashMap<>( + this.kq.getConsumerPositions(hdfsOffsets).size() + ); + for (Map.Entry entry : this.kq.getConsumerPositions(hdfsOffsets).entrySet()) { + serializedKafkaOffset.put(entry.getKey().toString(), entry.getValue()); // offset + } + kafkaOffset = new KafkaOffset(serializedKafkaOffset); } else { - kafkaOffset = new KafkaOffset(this.kq.getBeginningOffsets(null)); + Map serializedKafkaOffset = new HashMap<>(this.kq.getBeginningOffsets(null).size()); + for (Map.Entry entry : this.kq.getBeginningOffsets(null).entrySet()) { + serializedKafkaOffset.put(entry.getKey().toString(), entry.getValue()); // offset + } + kafkaOffset = new KafkaOffset(serializedKafkaOffset); } } else { - kafkaOffset = new KafkaOffset(this.kq.getBeginningOffsets(null)); + Map serializedKafkaOffset = new HashMap<>(this.kq.getBeginningOffsets(null).size()); + for (Map.Entry entry : this.kq.getBeginningOffsets(null).entrySet()) { + serializedKafkaOffset.put(entry.getKey().toString(), entry.getValue()); // offset + } + kafkaOffset = new KafkaOffset(serializedKafkaOffset); } } if (hdfsOffset.isStub() && longOffset == null && kafkaOffset.isStub()) { @@ -251,7 +265,11 @@ public Offset latestOffset() { longOffset = new LongOffset(this.aq.incrementAndGetLatestOffset()); } if (this.config.isKafkaEnabled) { - kafkaOffset = new KafkaOffset(this.kq.getInitialEndOffsets()); + Map serializedKafkaOffset = new HashMap<>(this.kq.getInitialEndOffsets().size()); + for (Map.Entry entry : this.kq.getInitialEndOffsets().entrySet()) { + serializedKafkaOffset.put(entry.getKey().toString(), entry.getValue()); // offset + } + kafkaOffset = new KafkaOffset(serializedKafkaOffset); } if (hdfsOffset.isStub() && longOffset == null && kafkaOffset.isStub()) { throw new IllegalStateException("no datasources enabled, can't get latest offset"); diff --git a/src/main/java/com/teragrep/pth_06/planner/offset/KafkaOffset.java b/src/main/java/com/teragrep/pth_06/planner/offset/KafkaOffset.java index 6865354d..b9ed8596 100644 --- a/src/main/java/com/teragrep/pth_06/planner/offset/KafkaOffset.java +++ b/src/main/java/com/teragrep/pth_06/planner/offset/KafkaOffset.java @@ -51,7 +51,6 @@ import org.apache.spark.sql.connector.read.streaming.Offset; import java.io.Serializable; -import java.lang.reflect.Type; import java.util.HashMap; import java.util.Map; @@ -63,9 +62,6 @@ */ public class KafkaOffset extends Offset implements Serializable { - private static final Type mapType = new TypeToken>() { - }.getType(); - private final Map serializedKafkaOffset; private final boolean stub; @@ -73,23 +69,18 @@ public KafkaOffset() { this(new HashMap<>(), true); } - public KafkaOffset(Map offset) { - this(offset, false); + public KafkaOffset(String s) { + this(new Gson().fromJson(s, new TypeToken>() { + }.getType()), false); } - public KafkaOffset(Map offset, boolean stub) { - serializedKafkaOffset = new HashMap<>(offset.size()); - for (Map.Entry entry : offset.entrySet()) { - - serializedKafkaOffset.put(entry.getKey().toString(), entry.getValue()); // offset - } - this.stub = stub; + public KafkaOffset(Map serializedKafkaOffset) { + this(serializedKafkaOffset, false); } - public KafkaOffset(String s) { - Gson gson = new Gson(); - serializedKafkaOffset = gson.fromJson(s, mapType); - stub = false; + public KafkaOffset(Map serializedKafkaOffset, boolean stub) { + this.serializedKafkaOffset = serializedKafkaOffset; + this.stub = stub; } public Map getOffsetMap() { diff --git a/src/test/java/com/teragrep/pth_06/planner/DatasourceOffsetTest.java b/src/test/java/com/teragrep/pth_06/planner/DatasourceOffsetTest.java index 45a9c978..01c289be 100644 --- a/src/test/java/com/teragrep/pth_06/planner/DatasourceOffsetTest.java +++ b/src/test/java/com/teragrep/pth_06/planner/DatasourceOffsetTest.java @@ -63,7 +63,11 @@ public void serdeTest() { LongOffset longOffset = new LongOffset(0L); Map topicPartitionLongMap = new HashMap<>(); topicPartitionLongMap.put(new TopicPartition("test", 0), 0L); - KafkaOffset kafkaOffset = new KafkaOffset(topicPartitionLongMap); + Map serializedKafkaOffset = new HashMap<>(topicPartitionLongMap.size()); + for (Map.Entry entry : topicPartitionLongMap.entrySet()) { + serializedKafkaOffset.put(entry.getKey().toString(), entry.getValue()); // offset + } + KafkaOffset kafkaOffset = new KafkaOffset(serializedKafkaOffset); DatasourceOffset datasourceOffset = new DatasourceOffset(longOffset, kafkaOffset); String ser = datasourceOffset.json(); @@ -77,7 +81,11 @@ public void serdeTest() { public void kafkaOffsetSerdeTest() { Map topicPartitionLongMap = new HashMap<>(); topicPartitionLongMap.put(new TopicPartition("test", 777), 9999L); - KafkaOffset kafkaOffset = new KafkaOffset(topicPartitionLongMap); + Map serializedKafkaOffset = new HashMap<>(topicPartitionLongMap.size()); + for (Map.Entry entry : topicPartitionLongMap.entrySet()) { + serializedKafkaOffset.put(entry.getKey().toString(), entry.getValue()); // offset + } + KafkaOffset kafkaOffset = new KafkaOffset(serializedKafkaOffset); String ser = kafkaOffset.json(); KafkaOffset deser = new KafkaOffset(ser); From 8d9d937000d6812f9a51aa2ce43ae68f26460acf Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Mon, 28 Oct 2024 15:09:11 +0200 Subject: [PATCH 52/62] Replaced returning of stub with exception throw. --- .../com/teragrep/pth_06/planner/offset/DatasourceOffset.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/com/teragrep/pth_06/planner/offset/DatasourceOffset.java b/src/main/java/com/teragrep/pth_06/planner/offset/DatasourceOffset.java index 5b0db417..ce3a6baa 100644 --- a/src/main/java/com/teragrep/pth_06/planner/offset/DatasourceOffset.java +++ b/src/main/java/com/teragrep/pth_06/planner/offset/DatasourceOffset.java @@ -108,7 +108,7 @@ public KafkaOffset getKafkaOffset() { KafkaOffset kafkaOffset = serializedDatasourceOffset.kafkaOffset; if (kafkaOffset.isStub() || kafkaOffset.getOffsetMap().isEmpty()) { - return kafkaOffset; + throw new RuntimeException("kafkaOffset must not be empty"); } for (Map.Entry entry : kafkaOffset.getOffsetMap().entrySet()) { From 3194415afbe13edf9b84d86b32d22f485276dd02 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Wed, 30 Oct 2024 12:44:50 +0200 Subject: [PATCH 53/62] Checkstyle plugin implementation and Checkstyle code cleanup. WIP --- pom.xml | 218 ++++++++++++++++++ .../teragrep/pth_06/FileSystemFactory.java | 2 +- .../teragrep/pth_06/TeragrepDatasource.java | 2 +- .../pth_06/config/ConditionConfig.java | 9 +- .../teragrep/pth_06/planner/ArchiveQuery.java | 8 +- .../pth_06/planner/BatchSizeLimit.java | 8 +- .../teragrep/pth_06/planner/HdfsQuery.java | 17 +- .../teragrep/pth_06/planner/KafkaQuery.java | 12 +- .../planner/MockKafkaConsumerFactory.java | 9 +- .../pth_06/planner/StreamDBClient.java | 4 +- .../planner/offset/DatasourceOffset.java | 2 +- .../planner/walker/ConditionWalker.java | 12 +- .../pth_06/planner/walker/EarliestWalker.java | 6 +- .../planner/walker/HdfsConditionWalker.java | 4 +- .../pth_06/planner/walker/KafkaWalker.java | 4 +- .../teragrep/pth_06/planner/walker/Util.java | 3 +- .../pth_06/planner/walker/XmlWalker.java | 10 +- .../walker/conditions/EarliestCondition.java | 9 +- .../walker/conditions/ElementCondition.java | 9 +- .../walker/conditions/HostCondition.java | 11 +- .../walker/conditions/IndexCondition.java | 11 +- .../conditions/IndexStatementCondition.java | 9 +- .../walker/conditions/LatestCondition.java | 9 +- .../walker/conditions/QueryCondition.java | 2 +- .../conditions/SourceTypeCondition.java | 11 +- .../teragrep/pth_06/scheduler/BatchSlice.java | 2 +- ...ArchiveMicroBatchInputPartitionReader.java | 3 +- .../teragrep/pth_06/task/hdfs/AvroRead.java | 6 +- .../task/hdfs/HdfsRecordConversion.java | 4 +- .../pth_06/task/s3/Pth06S3Client.java | 11 +- .../teragrep/pth_06/task/s3/RowConverter.java | 12 +- 31 files changed, 347 insertions(+), 92 deletions(-) diff --git a/pom.xml b/pom.xml index e635ac5c..d170ef50 100644 --- a/pom.xml +++ b/pom.xml @@ -520,6 +520,224 @@ true
+ + org.apache.maven.plugins + maven-checkstyle-plugin + 3.5.0 + + + + scan-errors + + check + + process-classes + + error + true + true + false + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + scan-warnings + + check + + process-classes + + warning + true + false + false + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
diff --git a/src/main/java/com/teragrep/pth_06/FileSystemFactory.java b/src/main/java/com/teragrep/pth_06/FileSystemFactory.java index d6f2680d..826925ca 100644 --- a/src/main/java/com/teragrep/pth_06/FileSystemFactory.java +++ b/src/main/java/com/teragrep/pth_06/FileSystemFactory.java @@ -51,6 +51,6 @@ public interface FileSystemFactory { - FileSystem fileSystem(boolean initializeUGI) throws IOException; + public abstract FileSystem fileSystem(boolean initializeUGI) throws IOException; } diff --git a/src/main/java/com/teragrep/pth_06/TeragrepDatasource.java b/src/main/java/com/teragrep/pth_06/TeragrepDatasource.java index 9066b5d9..2bfab1d5 100644 --- a/src/main/java/com/teragrep/pth_06/TeragrepDatasource.java +++ b/src/main/java/com/teragrep/pth_06/TeragrepDatasource.java @@ -99,7 +99,7 @@ public final class TeragrepDatasource implements DataSourceRegister, TableProvid }); @Override - public Table getTable(StructType schema, Transform[] partitioning, Map properties) { + public Table getTable(StructType schemaArg, Transform[] partitioning, Map properties) { return this; } diff --git a/src/main/java/com/teragrep/pth_06/config/ConditionConfig.java b/src/main/java/com/teragrep/pth_06/config/ConditionConfig.java index 618eaeed..eb023309 100644 --- a/src/main/java/com/teragrep/pth_06/config/ConditionConfig.java +++ b/src/main/java/com/teragrep/pth_06/config/ConditionConfig.java @@ -107,12 +107,15 @@ public boolean streamQuery() { @Override public boolean equals(Object object) { - if (this == object) + if (this == object) { return true; - if (object == null) + } + if (object == null) { return false; - if (object.getClass() != this.getClass()) + } + if (object.getClass() != this.getClass()) { return false; + } final ConditionConfig cast = (ConditionConfig) object; return this.bloomEnabled == cast.bloomEnabled && this.streamQuery == cast.streamQuery && this.withoutFilters == cast.withoutFilters && this.ctx == cast.ctx; diff --git a/src/main/java/com/teragrep/pth_06/planner/ArchiveQuery.java b/src/main/java/com/teragrep/pth_06/planner/ArchiveQuery.java index ea8240d1..45c0ceb1 100644 --- a/src/main/java/com/teragrep/pth_06/planner/ArchiveQuery.java +++ b/src/main/java/com/teragrep/pth_06/planner/ArchiveQuery.java @@ -59,14 +59,14 @@ */ public interface ArchiveQuery { - Result> processBetweenUnixEpochHours( + public abstract Result> processBetweenUnixEpochHours( long startHour, long endHour ); - void commit(long offset); + public abstract void commit(long offset); - Long getInitialOffset(); + public abstract Long getInitialOffset(); - Long incrementAndGetLatestOffset(); + public abstract Long incrementAndGetLatestOffset(); } diff --git a/src/main/java/com/teragrep/pth_06/planner/BatchSizeLimit.java b/src/main/java/com/teragrep/pth_06/planner/BatchSizeLimit.java index 95df9682..f262225b 100644 --- a/src/main/java/com/teragrep/pth_06/planner/BatchSizeLimit.java +++ b/src/main/java/com/teragrep/pth_06/planner/BatchSizeLimit.java @@ -52,16 +52,16 @@ final class BatchSizeLimit { /** Maximum weight of the batch (length of executor queue * amount of executors) */ - final long maxWeight; + private final long maxWeight; /** Maximum count of total objects, in case the weights of the objects are small */ - final long maxObjectCount; + private final long maxObjectCount; /** accumulated sum of weights */ - float accumulatedWeight; + private float accumulatedWeight; /** accumulated sum of object count */ - long accumulatedObjectCount; + private long accumulatedObjectCount; /** * Initialize the BatchSizeLimit with the given maximum size and maximum object count. diff --git a/src/main/java/com/teragrep/pth_06/planner/HdfsQuery.java b/src/main/java/com/teragrep/pth_06/planner/HdfsQuery.java index 27105699..32700760 100644 --- a/src/main/java/com/teragrep/pth_06/planner/HdfsQuery.java +++ b/src/main/java/com/teragrep/pth_06/planner/HdfsQuery.java @@ -53,17 +53,20 @@ public interface HdfsQuery { - LinkedList processBetweenHdfsFileMetadata(HdfsOffset startOffset, HdfsOffset endOffset); + public abstract LinkedList processBetweenHdfsFileMetadata( + HdfsOffset startOffset, + HdfsOffset endOffset + ); - void commit(HdfsOffset offset); + public abstract void commit(HdfsOffset offset); - JsonArray hdfsOffsetMapToJSON(); + public abstract JsonArray hdfsOffsetMapToJSON(); - HdfsOffset getBeginningOffsets(); + public abstract HdfsOffset getBeginningOffsets(); - HdfsOffset getInitialEndOffsets(); // Delete after testing incrementAndGetLatestOffset() functionality thoroughly. + public abstract HdfsOffset getInitialEndOffsets(); // Delete after testing incrementAndGetLatestOffset() functionality thoroughly. - HdfsOffset incrementAndGetLatestOffset(); // replaces getInitialEndOffsets() + public abstract HdfsOffset incrementAndGetLatestOffset(); // replaces getInitialEndOffsets() - boolean isStub(); + public abstract boolean isStub(); } diff --git a/src/main/java/com/teragrep/pth_06/planner/KafkaQuery.java b/src/main/java/com/teragrep/pth_06/planner/KafkaQuery.java index 88988eb6..2be65cdb 100644 --- a/src/main/java/com/teragrep/pth_06/planner/KafkaQuery.java +++ b/src/main/java/com/teragrep/pth_06/planner/KafkaQuery.java @@ -59,15 +59,15 @@ */ public interface KafkaQuery { - Map getInitialEndOffsets(); + public abstract Map getInitialEndOffsets(); - Map getEndOffsets(KafkaOffset startOffset); + public abstract Map getEndOffsets(KafkaOffset startOffset); - Map getBeginningOffsets(KafkaOffset endOffset); + public abstract Map getBeginningOffsets(KafkaOffset endOffset); - void commit(KafkaOffset offset); + public abstract void commit(KafkaOffset offset); - void seekToHdfsOffsets(JsonArray hdfsStartOffsets); + public abstract void seekToHdfsOffsets(JsonArray hdfsStartOffsets); - Map getConsumerPositions(JsonArray startOffsets); + public abstract Map getConsumerPositions(JsonArray startOffsets); } diff --git a/src/main/java/com/teragrep/pth_06/planner/MockKafkaConsumerFactory.java b/src/main/java/com/teragrep/pth_06/planner/MockKafkaConsumerFactory.java index 106308dd..9e08abac 100644 --- a/src/main/java/com/teragrep/pth_06/planner/MockKafkaConsumerFactory.java +++ b/src/main/java/com/teragrep/pth_06/planner/MockKafkaConsumerFactory.java @@ -57,6 +57,7 @@ import java.nio.charset.StandardCharsets; import java.util.Collections; import java.util.HashMap; +import java.util.Map; /** *

Mock Kafka Consumer Factory

Mocked Kafka Consumer factory used for testing. @@ -65,9 +66,9 @@ * @author Mikko Kortelainen */ @VisibleForTesting -public class MockKafkaConsumerFactory { +public final class MockKafkaConsumerFactory { - final static private Logger LOGGER = LoggerFactory.getLogger(MockKafkaConsumerFactory.class); + private static final Logger LOGGER = LoggerFactory.getLogger(MockKafkaConsumerFactory.class); private MockKafkaConsumerFactory() { @@ -251,13 +252,13 @@ public static Consumer getConsumer() { consumer = new MockConsumer(OffsetResetStrategy.EARLIEST); consumer.assign(Collections.singletonList(new TopicPartition("testConsumerTopic", 0))); - HashMap beginningOffsets = new HashMap<>(); + Map beginningOffsets = new HashMap<>(); beginningOffsets.put(new TopicPartition("testConsumerTopic", 0), 0L); // start is inclusive consumer.updateBeginningOffsets(beginningOffsets); generateEvents(consumer); - HashMap endOffsets = new HashMap<>(); + Map endOffsets = new HashMap<>(); endOffsets.put(new TopicPartition("testConsumerTopic", 0), 14L); // end is exclusive consumer.updateEndOffsets(endOffsets); LOGGER.debug(endOffsets.toString()); diff --git a/src/main/java/com/teragrep/pth_06/planner/StreamDBClient.java b/src/main/java/com/teragrep/pth_06/planner/StreamDBClient.java index ccae7ea6..b27233ad 100644 --- a/src/main/java/com/teragrep/pth_06/planner/StreamDBClient.java +++ b/src/main/java/com/teragrep/pth_06/planner/StreamDBClient.java @@ -336,7 +336,7 @@ public class NestedTopNQuery { logtimeFunction.as(logtime) }; - private Table getTableStatement(Condition journaldbCondition, Date day) { + private Table getTableStatement(Condition journaldbConditionArg, Date day) { SelectOnConditionStep selectOnConditionStep = select(resultFields) .from(GetArchivedObjectsFilterTable.FILTER_TABLE) @@ -359,7 +359,7 @@ private Table getTableStatement(Condition journaldbCondition, Date day) } return selectOnConditionStep - .where(JOURNALDB.LOGFILE.LOGDATE.eq(day).and(journaldbCondition)) + .where(JOURNALDB.LOGFILE.LOGDATE.eq(day).and(journaldbConditionArg)) .orderBy(logtimeForOrderBy, JOURNALDB.LOGFILE.ID.asc()) .asTable(innerTable); } diff --git a/src/main/java/com/teragrep/pth_06/planner/offset/DatasourceOffset.java b/src/main/java/com/teragrep/pth_06/planner/offset/DatasourceOffset.java index ce3a6baa..52407dcd 100644 --- a/src/main/java/com/teragrep/pth_06/planner/offset/DatasourceOffset.java +++ b/src/main/java/com/teragrep/pth_06/planner/offset/DatasourceOffset.java @@ -61,7 +61,7 @@ */ public class DatasourceOffset extends Offset implements Serializable { - SerializedDatasourceOffset serializedDatasourceOffset; + private final SerializedDatasourceOffset serializedDatasourceOffset; public DatasourceOffset(HdfsOffset hdfsOffset, KafkaOffset kafkaOffset) { this(hdfsOffset, null, kafkaOffset); diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/ConditionWalker.java b/src/main/java/com/teragrep/pth_06/planner/walker/ConditionWalker.java index ad85fa15..5f6cb3d9 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/ConditionWalker.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/ConditionWalker.java @@ -96,8 +96,8 @@ public ConditionWalker(DSLContext ctx, boolean bloomEnabled, boolean withoutFilt this.combinedMatchSet = new HashSet<>(); } - public Condition fromString(String inXml, boolean streamQuery) throws Exception { - this.streamQuery = streamQuery; + public Condition fromString(String inXml, boolean streamQueryInput) throws Exception { + this.streamQuery = streamQueryInput; return fromString(inXml); } @@ -115,13 +115,13 @@ public Condition emitLogicalOperation(String op, Object l, Object r) throws Exce if (op == null) { throw new Exception("Parse error, unbalanced elements. " + left.toString()); } - if (op.equalsIgnoreCase("AND")) { + if ("AND".equalsIgnoreCase(op)) { rv = left.and(right); } - else if (op.equalsIgnoreCase("OR")) { + else if ("OR".equalsIgnoreCase(op)) { rv = left.or(right); } - else if (op.equalsIgnoreCase("NOT")) { + else if ("NOT".equalsIgnoreCase(op)) { rv = left.not(); } else { @@ -143,7 +143,7 @@ public Condition emitUnaryOperation(String op, Element current) throws Exception throw new Exception("Parse error, op was null"); } if (rv != null) { - if (op.equalsIgnoreCase("NOT")) { + if ("NOT".equalsIgnoreCase(op)) { rv = rv.not(); } else { diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/EarliestWalker.java b/src/main/java/com/teragrep/pth_06/planner/walker/EarliestWalker.java index a20c5c66..a43ac14b 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/EarliestWalker.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/EarliestWalker.java @@ -75,7 +75,7 @@ Long emitElem(Element current) { long earliestEpoch = globalEarliestEpoch; - if (tag.equalsIgnoreCase("earliest") || tag.equalsIgnoreCase("index_earliest")) { + if ("earliest".equalsIgnoreCase(tag) || "index_earliest".equalsIgnoreCase(tag)) { earliestEpoch = Long.parseLong(value); @@ -97,7 +97,7 @@ public Long emitLogicalOperation(String op, Object l, Object r) throws Exception throw new Exception("Parse error, unbalanced elements. " + left); } - if (op.equalsIgnoreCase("AND") || op.equalsIgnoreCase("OR")) { + if ("AND".equalsIgnoreCase(op) || "OR".equalsIgnoreCase(op)) { rv = left < right ? left : right; } @@ -118,7 +118,7 @@ public Long emitUnaryOperation(String op, Element current) throws Exception { throw new Exception("Parse error op was null"); } if (rv != null) { - if (op.equalsIgnoreCase("NOT")) { + if ("NOT".equalsIgnoreCase(op)) { // Unary operations ignored rv = null; } diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/HdfsConditionWalker.java b/src/main/java/com/teragrep/pth_06/planner/walker/HdfsConditionWalker.java index 8cc40e0f..d292be50 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/HdfsConditionWalker.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/HdfsConditionWalker.java @@ -59,8 +59,8 @@ String emitElem(Element current) { String queryCondition = null; // only index equals supported - if (tag.equalsIgnoreCase("index")) { - if (operation.equalsIgnoreCase("EQUALS")) { + if ("index".equalsIgnoreCase(tag)) { + if ("EQUALS".equalsIgnoreCase(operation)) { queryCondition = GlobToRegEx.regexify(value); } } diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/KafkaWalker.java b/src/main/java/com/teragrep/pth_06/planner/walker/KafkaWalker.java index 8533f2bf..f179d6f8 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/KafkaWalker.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/KafkaWalker.java @@ -64,8 +64,8 @@ String emitElem(Element current) { String queryCondition = null; // only index equals supported - if (tag.equalsIgnoreCase("index")) { - if (operation.equalsIgnoreCase("EQUALS")) { + if ("index".equalsIgnoreCase(tag)) { + if ("EQUALS".equalsIgnoreCase(operation)) { queryCondition = GlobToRegEx.regexify(value); } } diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/Util.java b/src/main/java/com/teragrep/pth_06/planner/walker/Util.java index 55e6aedd..4be490cd 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/Util.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/Util.java @@ -106,8 +106,9 @@ public static long relativeTimeModifier(Timestamp timestamp, String value) { Instant rv = null; Matcher matcher = Pattern.compile("^-?\\d+").matcher(value); - if (!matcher.find()) + if (!matcher.find()) { throw new NumberFormatException("Unknown relative time modifier string [" + value + "]"); + } long v = Long.parseLong(matcher.group()); matcher.group().length(); diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/XmlWalker.java b/src/main/java/com/teragrep/pth_06/planner/walker/XmlWalker.java index dce9a1fc..f6267839 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/XmlWalker.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/XmlWalker.java @@ -166,7 +166,7 @@ else if (rht == null) { } else { // leaf - if (op != null && op.equals("NOT")) { + if (op != null && "NOT".equals(op)) { LOGGER.debug("Emit Unary operation op:" + op + " l:" + current); rv = emitUnaryOperation(op, current); } @@ -218,15 +218,17 @@ else if (rht == null) { */ public static String escapeSpecialCharacters(String input) { final List specialCharacters = Arrays.asList("\\", "*", "+", "?", "%"); - if (input.equals("*")) { + if ("*".equals(input)) { return "%"; } else { return Arrays.stream(input.split("")).map((c) -> { - if (specialCharacters.contains(c)) + if (specialCharacters.contains(c)) { return "\\" + c; - else + } + else { return c; + } }).collect(Collectors.joining()); } } diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/EarliestCondition.java b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/EarliestCondition.java index 80ddd428..2772bf67 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/EarliestCondition.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/EarliestCondition.java @@ -87,12 +87,15 @@ public Condition condition() { @Override public boolean equals(final Object object) { - if (this == object) + if (this == object) { return true; - if (object == null) + } + if (object == null) { return false; - if (object.getClass() != this.getClass()) + } + if (object.getClass() != this.getClass()) { return false; + } final EarliestCondition cast = (EarliestCondition) object; return this.value.equals(cast.value); } diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/ElementCondition.java b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/ElementCondition.java index 4faad3f6..a33f068e 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/ElementCondition.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/ElementCondition.java @@ -132,12 +132,15 @@ public Set> requiredTables() { @Override public boolean equals(final Object object) { - if (this == object) + if (this == object) { return true; - if (object == null) + } + if (object == null) { return false; - if (object.getClass() != this.getClass()) + } + if (object.getClass() != this.getClass()) { return false; + } final ElementCondition cast = (ElementCondition) object; return this.element.equals(cast.element) && this.config.equals(cast.config); } diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/HostCondition.java b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/HostCondition.java index d79d836a..cc1b0ece 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/HostCondition.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/HostCondition.java @@ -70,7 +70,7 @@ public Condition condition() { else { condition = StreamDBClient.GetArchivedObjectsFilterTable.host.like(value.replace('*', '%').toLowerCase()); } - if (operation.equalsIgnoreCase("NOT_EQUALS")) { + if ("NOT_EQUALS".equalsIgnoreCase(operation)) { condition = condition.not(); } return condition; @@ -78,12 +78,15 @@ public Condition condition() { @Override public boolean equals(final Object object) { - if (this == object) + if (this == object) { return true; - if (object == null) + } + if (object == null) { return false; - if (object.getClass() != this.getClass()) + } + if (object.getClass() != this.getClass()) { return false; + } final HostCondition cast = (HostCondition) object; return this.streamQuery == cast.streamQuery && this.value.equals(cast.value) && this.operation.equals(cast.operation); diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/IndexCondition.java b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/IndexCondition.java index a49a1951..72e208e4 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/IndexCondition.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/IndexCondition.java @@ -71,7 +71,7 @@ public Condition condition() { condition = StreamDBClient.GetArchivedObjectsFilterTable.directory .like(value.replace('*', '%').toLowerCase()); } - if (operation.equalsIgnoreCase("NOT_EQUALS")) { + if ("NOT_EQUALS".equalsIgnoreCase(operation)) { condition = condition.not(); } return condition; @@ -79,12 +79,15 @@ public Condition condition() { @Override public boolean equals(final Object object) { - if (this == object) + if (this == object) { return true; - if (object == null) + } + if (object == null) { return false; - if (object.getClass() != this.getClass()) + } + if (object.getClass() != this.getClass()) { return false; + } final IndexCondition cast = (IndexCondition) object; return this.streamQuery == cast.streamQuery && this.value.equals(cast.value) && this.operation.equals(cast.operation); diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/IndexStatementCondition.java b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/IndexStatementCondition.java index c8fb9505..54b3def9 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/IndexStatementCondition.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/IndexStatementCondition.java @@ -140,12 +140,15 @@ public Set> requiredTables() { @Override public boolean equals(final Object object) { - if (this == object) + if (this == object) { return true; - if (object == null) + } + if (object == null) { return false; - if (object.getClass() != this.getClass()) + } + if (object.getClass() != this.getClass()) { return false; + } final IndexStatementCondition cast = (IndexStatementCondition) object; return this.value.equals(cast.value) && this.config.equals(cast.config); } diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/LatestCondition.java b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/LatestCondition.java index d11a7781..ac3f4310 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/LatestCondition.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/LatestCondition.java @@ -84,12 +84,15 @@ public Condition condition() { @Override public boolean equals(final Object object) { - if (this == object) + if (this == object) { return true; - if (object == null) + } + if (object == null) { return false; - if (object.getClass() != this.getClass()) + } + if (object.getClass() != this.getClass()) { return false; + } final LatestCondition cast = (LatestCondition) object; return this.value.equals(cast.value); } diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/QueryCondition.java b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/QueryCondition.java index b9dec429..a71064a4 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/QueryCondition.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/QueryCondition.java @@ -49,5 +49,5 @@ public interface QueryCondition { - Condition condition(); + public abstract Condition condition(); } diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/SourceTypeCondition.java b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/SourceTypeCondition.java index 2c3c9aee..2b2880d9 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/SourceTypeCondition.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/SourceTypeCondition.java @@ -70,7 +70,7 @@ public Condition condition() { else { condition = StreamDBClient.GetArchivedObjectsFilterTable.stream.like(value.replace('*', '%').toLowerCase()); } - if (operation.equalsIgnoreCase("NOT_EQUALS")) { + if ("NOT_EQUALS".equalsIgnoreCase(operation)) { condition = condition.not(); } return condition; @@ -78,12 +78,15 @@ public Condition condition() { @Override public boolean equals(final Object object) { - if (this == object) + if (this == object) { return true; - if (object == null) + } + if (object == null) { return false; - if (object.getClass() != this.getClass()) + } + if (object.getClass() != this.getClass()) { return false; + } final SourceTypeCondition cast = (SourceTypeCondition) object; return this.streamQuery == cast.streamQuery && this.value.equals(cast.value) && this.operation.equals(cast.operation); diff --git a/src/main/java/com/teragrep/pth_06/scheduler/BatchSlice.java b/src/main/java/com/teragrep/pth_06/scheduler/BatchSlice.java index 56905171..6dc9fdfa 100644 --- a/src/main/java/com/teragrep/pth_06/scheduler/BatchSlice.java +++ b/src/main/java/com/teragrep/pth_06/scheduler/BatchSlice.java @@ -62,7 +62,7 @@ */ public final class BatchSlice implements Serializable { - public enum Type { + public static enum Type { ARCHIVE, KAFKA, HDFS } diff --git a/src/main/java/com/teragrep/pth_06/task/ArchiveMicroBatchInputPartitionReader.java b/src/main/java/com/teragrep/pth_06/task/ArchiveMicroBatchInputPartitionReader.java index 1fc45a46..6920112d 100644 --- a/src/main/java/com/teragrep/pth_06/task/ArchiveMicroBatchInputPartitionReader.java +++ b/src/main/java/com/teragrep/pth_06/task/ArchiveMicroBatchInputPartitionReader.java @@ -186,8 +186,9 @@ public InternalRow get() { @Override public void close() throws IOException { - if (LOGGER.isDebugEnabled()) + if (LOGGER.isDebugEnabled()) { LOGGER.debug("ArchiveMicroBatchInputPartitionReader.close>"); + } if (rowConverter != null) { rowConverter.close(); } diff --git a/src/main/java/com/teragrep/pth_06/task/hdfs/AvroRead.java b/src/main/java/com/teragrep/pth_06/task/hdfs/AvroRead.java index ac3f4390..07e31d77 100644 --- a/src/main/java/com/teragrep/pth_06/task/hdfs/AvroRead.java +++ b/src/main/java/com/teragrep/pth_06/task/hdfs/AvroRead.java @@ -51,10 +51,10 @@ public interface AvroRead { - boolean next(); + public abstract boolean next(); - SyslogRecord record(); + public abstract SyslogRecord record(); - void close() throws IOException; + public abstract void close() throws IOException; } diff --git a/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversion.java b/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversion.java index f14a2ab8..97914134 100644 --- a/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversion.java +++ b/src/main/java/com/teragrep/pth_06/task/hdfs/HdfsRecordConversion.java @@ -51,8 +51,8 @@ public interface HdfsRecordConversion { - boolean next() throws IOException; + public abstract boolean next() throws IOException; - InternalRow row(); + public abstract InternalRow row(); } diff --git a/src/main/java/com/teragrep/pth_06/task/s3/Pth06S3Client.java b/src/main/java/com/teragrep/pth_06/task/s3/Pth06S3Client.java index 30e7c0e1..bf777805 100644 --- a/src/main/java/com/teragrep/pth_06/task/s3/Pth06S3Client.java +++ b/src/main/java/com/teragrep/pth_06/task/s3/Pth06S3Client.java @@ -92,11 +92,11 @@ public Pth06S3Client(String S3endPoint, String S3identity, String S3credential) /** * Defines which signing algorithm is used. Default is V2. * - * @param signerKind Signing algorithm kind. + * @param signerKindInput Signing algorithm kind. * @return Reference to this builder. */ - public Pth06S3Client withSigner(SignerKind signerKind) { - this.signerKind = signerKind; + public Pth06S3Client withSigner(SignerKind signerKindInput) { + this.signerKind = signerKindInput; return this; } @@ -106,8 +106,9 @@ public Pth06S3Client withSigner(SignerKind signerKind) { * @return AmazonS3Client using the builder's values. */ public AmazonS3 build() { - if (LOGGER.isDebugEnabled()) + if (LOGGER.isDebugEnabled()) { LOGGER.info("Building S3APIWrapper for endpoint {}.", S3endPoint); + } if (StringUtils.isBlank(this.S3endPoint)) { throw new IllegalStateException("S3 endpoint is required."); @@ -168,7 +169,7 @@ public AmazonS3 build() { /** * S3 signing algorithm kinds. */ - public enum SignerKind { + public static enum SignerKind { // These constants are from com.amazonaws.services.s3.AmazonS3Client. Constants are private so cannot be used // here directly. Values select the authorization signing algorithm. V2 has to be used with TOS, V4 with AWS. diff --git a/src/main/java/com/teragrep/pth_06/task/s3/RowConverter.java b/src/main/java/com/teragrep/pth_06/task/s3/RowConverter.java index aaa8b766..fd45fa3f 100644 --- a/src/main/java/com/teragrep/pth_06/task/s3/RowConverter.java +++ b/src/main/java/com/teragrep/pth_06/task/s3/RowConverter.java @@ -129,8 +129,9 @@ public RowConverter( this.stream = UTF8String.fromString(stream.toLowerCase()); this.host = UTF8String.fromString(host.toLowerCase()); - if (LOGGER.isDebugEnabled()) + if (LOGGER.isDebugEnabled()) { LOGGER.debug("RowConverter> created with partition:" + this.bucket + " path: " + this.path); + } this.rowWriter = new UnsafeRowWriter(11); @@ -138,8 +139,9 @@ public RowConverter( this.skipNonRFC5424Files = skipNonRFC5424Files; - if (LOGGER.isDebugEnabled()) + if (LOGGER.isDebugEnabled()) { LOGGER.info("Initialized s3client:" + s3client); + } // initial status this.currentOffset = 0L; @@ -246,12 +248,13 @@ static long rfc3339ToEpoch(ZonedDateTime zonedDateTime) { public InternalRow get() { //System.out.println("RowConverter.get>"); - if (LOGGER.isDebugEnabled()) + if (LOGGER.isDebugEnabled()) { LOGGER .debug( "RowConverter.get> Partition (" + this.id + "):" + bucket + "/" + path + " Get(" + currentOffset + ")" ); + } if (LOGGER.isDebugEnabled()) { LOGGER.debug("Parsersyslog event:" + rfc5424Frame.toString()); @@ -285,8 +288,9 @@ public InternalRow get() { rowWriter.write(7, currentOffset); rowWriter.write(8, UTF8String.fromBytes(origin)); - if (LOGGER.isDebugEnabled()) + if (LOGGER.isDebugEnabled()) { LOGGER.debug("Get Event, row=written"); + } auditPlugin .audit( From b0461c1d62e9ed3761b1b5b8a414c54da71eb2b2 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Fri, 1 Nov 2024 15:49:34 +0200 Subject: [PATCH 54/62] Checkstyle plugin implementation and Checkstyle code cleanup, WIP. Added missing hashCode() overrides and other minor fixes. --- .../pth_06/config/ConditionConfig.java | 12 ++++++++++++ .../pth_06/planner/HdfsQueryProcessor.java | 2 +- .../teragrep/pth_06/planner/walker/Util.java | 10 ++++++++++ .../walker/conditions/EarliestCondition.java | 9 +++++++++ .../walker/conditions/ElementCondition.java | 18 ++++++++++++++++++ .../walker/conditions/HostCondition.java | 12 ++++++++++++ .../walker/conditions/IndexCondition.java | 13 +++++++++++++ .../conditions/IndexStatementCondition.java | 16 ++++++++++++++++ .../walker/conditions/LatestCondition.java | 9 +++++++++ .../walker/conditions/SourceTypeCondition.java | 13 +++++++++++++ 10 files changed, 113 insertions(+), 1 deletion(-) diff --git a/src/main/java/com/teragrep/pth_06/config/ConditionConfig.java b/src/main/java/com/teragrep/pth_06/config/ConditionConfig.java index eb023309..20ddc77d 100644 --- a/src/main/java/com/teragrep/pth_06/config/ConditionConfig.java +++ b/src/main/java/com/teragrep/pth_06/config/ConditionConfig.java @@ -120,4 +120,16 @@ public boolean equals(Object object) { return this.bloomEnabled == cast.bloomEnabled && this.streamQuery == cast.streamQuery && this.withoutFilters == cast.withoutFilters && this.ctx == cast.ctx; } + + @Override + public int hashCode() { + int result = 17; + result = 31 * result + (bloomEnabled ? 1 : 0); + result = 31 * result + (streamQuery ? 1 : 0); + result = 31 * result + (withoutFilters ? 1 : 0); + if (ctx != null) { + result = 31 * result + ctx.hashCode(); + } + return result; + } } diff --git a/src/main/java/com/teragrep/pth_06/planner/HdfsQueryProcessor.java b/src/main/java/com/teragrep/pth_06/planner/HdfsQueryProcessor.java index 0eb04580..97bdcdd8 100644 --- a/src/main/java/com/teragrep/pth_06/planner/HdfsQueryProcessor.java +++ b/src/main/java/com/teragrep/pth_06/planner/HdfsQueryProcessor.java @@ -111,7 +111,7 @@ public HdfsQueryProcessor(Config config) { topicsRegexString // topicsRegexString only searches for the given topic/topics (aka. directories). ); } - catch (Exception e) { + catch (IOException e) { throw new RuntimeException(e); } diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/Util.java b/src/main/java/com/teragrep/pth_06/planner/walker/Util.java index 4be490cd..ab49836a 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/Util.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/Util.java @@ -45,6 +45,8 @@ */ package com.teragrep.pth_06.planner.walker; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.w3c.dom.Element; import javax.xml.transform.*; @@ -68,6 +70,8 @@ */ public class Util { + private static final Logger LOGGER = LoggerFactory.getLogger(Util.class); + public static String stripQuotes(String value) { Matcher m = Pattern.compile("^\"(.*)\"$").matcher(value); Matcher m1 = Pattern.compile("^'(.*)'$").matcher(value); @@ -214,8 +218,14 @@ public static String elementAsString(Element el) { str = buffer.toString(); } catch (TransformerConfigurationException tex) { + if (LOGGER.isDebugEnabled()) { + LOGGER.debug(tex.getMessage(), tex); + } } catch (TransformerException ex) { + if (LOGGER.isDebugEnabled()) { + LOGGER.debug(ex.getMessage()); + } } return str; } diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/EarliestCondition.java b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/EarliestCondition.java index 2772bf67..30319afc 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/EarliestCondition.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/EarliestCondition.java @@ -99,4 +99,13 @@ public boolean equals(final Object object) { final EarliestCondition cast = (EarliestCondition) object; return this.value.equals(cast.value); } + + @Override + public int hashCode() { + int result = 17; + if (value != null) { + result = 31 * result + value.hashCode(); + } + return result; + } } diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/ElementCondition.java b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/ElementCondition.java index a33f068e..cb3c2b6c 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/ElementCondition.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/ElementCondition.java @@ -144,4 +144,22 @@ public boolean equals(final Object object) { final ElementCondition cast = (ElementCondition) object; return this.element.equals(cast.element) && this.config.equals(cast.config); } + + @Override + public int hashCode() { + int result = 17; + if (element.getTagName() != null) { + result = 31 * result + element.getTagName().hashCode(); + } + if (element.getAttribute("operation") != null) { + result = 31 * result + element.getAttribute("operation").hashCode(); + } + if (element.getAttribute("value") != null) { + result = 31 * result + element.getAttribute("value").hashCode(); + } + if (config != null) { + result = 31 * result + config.hashCode(); + } + return result; + } } diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/HostCondition.java b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/HostCondition.java index cc1b0ece..d2632405 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/HostCondition.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/HostCondition.java @@ -91,4 +91,16 @@ public boolean equals(final Object object) { return this.streamQuery == cast.streamQuery && this.value.equals(cast.value) && this.operation.equals(cast.operation); } + + @Override + public int hashCode() { + int result = 17; + if (value != null) { + result = 31 * result + value.hashCode(); + } + if (operation != null) { + result = 31 * result + operation.hashCode(); + } + return result; + } } diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/IndexCondition.java b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/IndexCondition.java index 72e208e4..397d3322 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/IndexCondition.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/IndexCondition.java @@ -92,4 +92,17 @@ public boolean equals(final Object object) { return this.streamQuery == cast.streamQuery && this.value.equals(cast.value) && this.operation.equals(cast.operation); } + + @Override + public int hashCode() { + int result = 17; + if (value != null) { + result = 31 * result + value.hashCode(); + } + if (operation != null) { + result = 31 * result + operation.hashCode(); + } + result = 31 * result + (streamQuery ? 1 : 0); + return result; + } } diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/IndexStatementCondition.java b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/IndexStatementCondition.java index 54b3def9..f86838f8 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/IndexStatementCondition.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/IndexStatementCondition.java @@ -152,4 +152,20 @@ public boolean equals(final Object object) { final IndexStatementCondition cast = (IndexStatementCondition) object; return this.value.equals(cast.value) && this.config.equals(cast.config); } + + @Override + public int hashCode() { + int result = 17; + if (value != null) { + result = 31 * result + value.hashCode(); + } + if (tokenizer != null) { + result = 31 * result + tokenizer.hashCode(); + } + if (config != null) { + result = 31 * result + config.hashCode(); + } + return result; + } + } diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/LatestCondition.java b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/LatestCondition.java index ac3f4310..7fa16188 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/LatestCondition.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/LatestCondition.java @@ -96,4 +96,13 @@ public boolean equals(final Object object) { final LatestCondition cast = (LatestCondition) object; return this.value.equals(cast.value); } + + @Override + public int hashCode() { + int result = 17; + if (value != null) { + result = 31 * result + value.hashCode(); + } + return result; + } } diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/SourceTypeCondition.java b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/SourceTypeCondition.java index 2b2880d9..dc58be82 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/SourceTypeCondition.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/SourceTypeCondition.java @@ -91,4 +91,17 @@ public boolean equals(final Object object) { return this.streamQuery == cast.streamQuery && this.value.equals(cast.value) && this.operation.equals(cast.operation); } + + @Override + public int hashCode() { + int result = 17; + if (value != null) { + result = 31 * result + value.hashCode(); + } + if (operation != null) { + result = 31 * result + operation.hashCode(); + } + result = 31 * result + (streamQuery ? 1 : 0); + return result; + } } From 9060b6be706136a6a6c11ead65386088a0cc0e5b Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Mon, 4 Nov 2024 10:26:33 +0200 Subject: [PATCH 55/62] Checkstyle plugin implementation and Checkstyle code cleanup, WIP. Added missing default keywords to switches. --- src/main/java/com/teragrep/pth_06/planner/walker/XmlWalker.java | 1 + .../pth_06/planner/walker/conditions/ElementCondition.java | 2 ++ 2 files changed, 3 insertions(+) diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/XmlWalker.java b/src/main/java/com/teragrep/pth_06/planner/walker/XmlWalker.java index f6267839..7b13ff1d 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/XmlWalker.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/XmlWalker.java @@ -162,6 +162,7 @@ else if (rht == null) { } break; } + default: } } else { diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/ElementCondition.java b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/ElementCondition.java index cb3c2b6c..d4e51a25 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/ElementCondition.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/ElementCondition.java @@ -92,6 +92,8 @@ public Condition condition() { QueryCondition host = new HostCondition(value, operation, config.streamQuery()); condition = host.condition(); break; + default: + LOGGER.debug("Element tag was not index, sourcetype or host: <{}>", tag); } if (!config.streamQuery()) { // Handle also time qualifiers From f8c9c7a772cf7e91fec47928c10a309e72c14849 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Mon, 4 Nov 2024 12:18:22 +0200 Subject: [PATCH 56/62] Checkstyle plugin implementation and Checkstyle code cleanup, WIP. Refactored Exceptions to conform to Checkstyle standards. --- .../pth_06/planner/ArchiveQueryProcessor.java | 5 ++++- .../pth_06/planner/HdfsQueryProcessor.java | 4 +++- .../pth_06/planner/KafkaQueryProcessor.java | 5 ++++- .../pth_06/planner/StreamDBClient.java | 5 ++++- .../planner/walker/ConditionWalker.java | 19 ++++++++++++------- .../pth_06/planner/walker/EarliestWalker.java | 16 ++++++++++------ .../planner/walker/HdfsConditionWalker.java | 10 +++++++--- .../pth_06/planner/walker/KafkaWalker.java | 8 ++++++-- .../pth_06/planner/walker/PlainWalker.java | 8 ++++---- .../pth_06/planner/walker/XmlWalker.java | 18 ++++++++++-------- 10 files changed, 64 insertions(+), 34 deletions(-) diff --git a/src/main/java/com/teragrep/pth_06/planner/ArchiveQueryProcessor.java b/src/main/java/com/teragrep/pth_06/planner/ArchiveQueryProcessor.java index 86ded566..0950851a 100644 --- a/src/main/java/com/teragrep/pth_06/planner/ArchiveQueryProcessor.java +++ b/src/main/java/com/teragrep/pth_06/planner/ArchiveQueryProcessor.java @@ -51,7 +51,10 @@ import org.jooq.types.ULong; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.xml.sax.SAXException; +import javax.xml.parsers.ParserConfigurationException; +import java.io.IOException; import java.sql.Date; import java.sql.SQLException; import java.time.Instant; @@ -102,7 +105,7 @@ public ArchiveQueryProcessor(Config config) { rollingDay = Instant.ofEpochSecond(this.earliestEpoch).atZone(ZoneId.systemDefault()).toLocalDate(); } - catch (Exception ex) { + catch (ParserConfigurationException | SAXException | IOException ex) { throw new RuntimeException( "ArchiveQueryProcessor problems when construction Query conditions query:" + config.query + " exception:" + ex diff --git a/src/main/java/com/teragrep/pth_06/planner/HdfsQueryProcessor.java b/src/main/java/com/teragrep/pth_06/planner/HdfsQueryProcessor.java index 97bdcdd8..1f1bdd53 100644 --- a/src/main/java/com/teragrep/pth_06/planner/HdfsQueryProcessor.java +++ b/src/main/java/com/teragrep/pth_06/planner/HdfsQueryProcessor.java @@ -54,7 +54,9 @@ import org.apache.kafka.common.TopicPartition; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.xml.sax.SAXException; +import javax.xml.parsers.ParserConfigurationException; import java.io.IOException; import java.util.HashMap; import java.util.Iterator; @@ -96,7 +98,7 @@ public HdfsQueryProcessor(Config config) { HdfsConditionWalker parser = new HdfsConditionWalker(); topicsRegexString = parser.fromString(config.query); } - catch (Exception e) { + catch (ParserConfigurationException | SAXException | IOException e) { throw new RuntimeException( "HdfsQueryProcessor problems when construction Query conditions query:" + config.query + " exception:" + e diff --git a/src/main/java/com/teragrep/pth_06/planner/KafkaQueryProcessor.java b/src/main/java/com/teragrep/pth_06/planner/KafkaQueryProcessor.java index 4e2d5c9f..8cc825e2 100644 --- a/src/main/java/com/teragrep/pth_06/planner/KafkaQueryProcessor.java +++ b/src/main/java/com/teragrep/pth_06/planner/KafkaQueryProcessor.java @@ -59,7 +59,10 @@ import org.apache.kafka.common.errors.TopicAuthorizationException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.xml.sax.SAXException; +import javax.xml.parsers.ParserConfigurationException; +import java.io.IOException; import java.time.Duration; import java.util.*; import java.util.regex.Matcher; @@ -94,7 +97,7 @@ public KafkaQueryProcessor(Config config) { KafkaWalker parser = new KafkaWalker(); topicsRegexString = parser.fromString(config.query); } - catch (Exception ex) { + catch (ParserConfigurationException | SAXException | IOException ex) { ex.printStackTrace(); throw new RuntimeException( "KafkaQueryProcessor problems when construction Query conditions query:" + config.query diff --git a/src/main/java/com/teragrep/pth_06/planner/StreamDBClient.java b/src/main/java/com/teragrep/pth_06/planner/StreamDBClient.java index b27233ad..e28b6829 100644 --- a/src/main/java/com/teragrep/pth_06/planner/StreamDBClient.java +++ b/src/main/java/com/teragrep/pth_06/planner/StreamDBClient.java @@ -61,6 +61,9 @@ import org.jooq.types.UShort; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.xml.sax.SAXException; + +import javax.xml.parsers.ParserConfigurationException; import static com.teragrep.pth_06.jooq.generated.streamdb.Streamdb.STREAMDB; import static com.teragrep.pth_06.jooq.generated.journaldb.Journaldb.JOURNALDB; @@ -133,7 +136,7 @@ public StreamDBClient(Config config) throws SQLException { streamdbCondition = walker.fromString(config.query, true); this.journaldbCondition = walker.fromString(config.query, false); } - catch (Exception e) { + catch (ParserConfigurationException | SAXException | IOException e) { throw new IllegalArgumentException(e); } diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/ConditionWalker.java b/src/main/java/com/teragrep/pth_06/planner/walker/ConditionWalker.java index 5f6cb3d9..9e086bc8 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/ConditionWalker.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/ConditionWalker.java @@ -54,6 +54,10 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.w3c.dom.Element; +import org.xml.sax.SAXException; + +import javax.xml.parsers.ParserConfigurationException; +import java.io.IOException; import java.util.HashSet; import java.util.Set; @@ -96,7 +100,8 @@ public ConditionWalker(DSLContext ctx, boolean bloomEnabled, boolean withoutFilt this.combinedMatchSet = new HashSet<>(); } - public Condition fromString(String inXml, boolean streamQueryInput) throws Exception { + public Condition fromString(String inXml, boolean streamQueryInput) + throws IllegalStateException, ParserConfigurationException, IOException, SAXException { this.streamQuery = streamQueryInput; return fromString(inXml); } @@ -107,13 +112,13 @@ public Set> conditionRequiredTables() { } @Override - public Condition emitLogicalOperation(String op, Object l, Object r) throws Exception { + public Condition emitLogicalOperation(String op, Object l, Object r) throws IllegalArgumentException { Condition rv; Condition left = (Condition) l; Condition right = (Condition) r; if (op == null) { - throw new Exception("Parse error, unbalanced elements. " + left.toString()); + throw new IllegalArgumentException("Parse error, unbalanced elements. " + left.toString()); } if ("AND".equalsIgnoreCase(op)) { rv = left.and(right); @@ -125,7 +130,7 @@ else if ("NOT".equalsIgnoreCase(op)) { rv = left.not(); } else { - throw new Exception( + throw new IllegalArgumentException( "Parse error, unssorted logical operation. op:" + op + " expression:" + left.toString() ); } @@ -133,21 +138,21 @@ else if ("NOT".equalsIgnoreCase(op)) { } @Override - public Condition emitUnaryOperation(String op, Element current) throws Exception { + public Condition emitUnaryOperation(String op, Element current) throws IllegalArgumentException { Condition rv = emitElem(current); LOGGER.info("ConditionWalker.emitUnaryOperation incoming op:" + op + " element:" + current); if (op == null) { - throw new Exception("Parse error, op was null"); + throw new IllegalArgumentException("Parse error, op was null"); } if (rv != null) { if ("NOT".equalsIgnoreCase(op)) { rv = rv.not(); } else { - throw new Exception( + throw new IllegalArgumentException( "Parse error, unsupported logical operation. op:" + op + " expression:" + rv.toString() ); } diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/EarliestWalker.java b/src/main/java/com/teragrep/pth_06/planner/walker/EarliestWalker.java index a43ac14b..ecca54ea 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/EarliestWalker.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/EarliestWalker.java @@ -88,13 +88,13 @@ Long emitElem(Element current) { } @Override - public Long emitLogicalOperation(String op, Object l, Object r) throws Exception { + public Long emitLogicalOperation(String op, Object l, Object r) throws IllegalStateException { Long rv; Long left = (Long) l; Long right = (Long) r; if (op == null) { - throw new Exception("Parse error, unbalanced elements. " + left); + throw new IllegalStateException("Parse error, unbalanced elements. " + left); } if ("AND".equalsIgnoreCase(op) || "OR".equalsIgnoreCase(op)) { @@ -102,20 +102,22 @@ public Long emitLogicalOperation(String op, Object l, Object r) throws Exception } else { - throw new Exception("Parse error, unsorted logical operation. op: " + op + " expression: " + left); + throw new IllegalStateException( + "Parse error, unsorted logical operation. op: " + op + " expression: " + left + ); } return rv; } @Override - public Long emitUnaryOperation(String op, Element current) throws Exception { + public Long emitUnaryOperation(String op, Element current) throws IllegalStateException { Long rv = emitElem(current); LOGGER.info("EarliestWalker.emitUnaryOperation incoming op: " + op + " element: " + current); if (op == null) { - throw new Exception("Parse error op was null"); + throw new IllegalStateException("Parse error op was null"); } if (rv != null) { if ("NOT".equalsIgnoreCase(op)) { @@ -123,7 +125,9 @@ public Long emitUnaryOperation(String op, Element current) throws Exception { rv = null; } else { - throw new Exception("Parse error, unsupported logical operation: " + op + " expression: " + rv); + throw new IllegalStateException( + "Parse error, unsupported logical operation: " + op + " expression: " + rv + ); } } diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/HdfsConditionWalker.java b/src/main/java/com/teragrep/pth_06/planner/walker/HdfsConditionWalker.java index d292be50..55efe709 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/HdfsConditionWalker.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/HdfsConditionWalker.java @@ -47,6 +47,10 @@ import com.teragrep.jue_01.GlobToRegEx; import org.w3c.dom.Element; +import org.xml.sax.SAXException; + +import javax.xml.parsers.ParserConfigurationException; +import java.io.IOException; // HDFS planner walker, only aims to filter out the topics as the planner only outputs the metadata for AVRO-files containing the records. The rest of the condition handling is done in the separate tasker walker. public class HdfsConditionWalker extends XmlWalker { @@ -67,12 +71,12 @@ String emitElem(Element current) { return queryCondition; } - public String fromString(String inXml) throws Exception { + public String fromString(String inXml) throws ParserConfigurationException, IOException, SAXException { return super.fromString(inXml); } @Override - String emitLogicalOperation(String op, Object l, Object r) throws Exception { + String emitLogicalOperation(String op, Object l, Object r) throws IllegalStateException { String left = (String) l; String right = (String) r; @@ -88,7 +92,7 @@ String emitLogicalOperation(String op, Object l, Object r) throws Exception { } @Override - String emitUnaryOperation(String op, Element current) throws Exception { + String emitUnaryOperation(String op, Element current) throws IllegalStateException { // NOT is a filter, not a topic matcher return null; } diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/KafkaWalker.java b/src/main/java/com/teragrep/pth_06/planner/walker/KafkaWalker.java index f179d6f8..51997989 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/KafkaWalker.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/KafkaWalker.java @@ -47,6 +47,10 @@ import com.teragrep.jue_01.GlobToRegEx; import org.w3c.dom.Element; +import org.xml.sax.SAXException; + +import javax.xml.parsers.ParserConfigurationException; +import java.io.IOException; /** *

Kafka Walker

@@ -72,12 +76,12 @@ String emitElem(Element current) { return queryCondition; } - public String fromString(String inXml) throws Exception { + public String fromString(String inXml) throws ParserConfigurationException, IOException, SAXException { return super.fromString(inXml); } @Override - public String emitLogicalOperation(String op, Object l, Object r) throws Exception { + public String emitLogicalOperation(String op, Object l, Object r) { String left = (String) l; String right = (String) r; diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/PlainWalker.java b/src/main/java/com/teragrep/pth_06/planner/walker/PlainWalker.java index 15c144d9..36769a70 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/PlainWalker.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/PlainWalker.java @@ -81,12 +81,12 @@ public String traverse(TreeWalker walker, String indent) { private String indent = "\t"; @Override - public Element emitLogicalOperation(String op, Object l, Object r) throws Exception { + public Element emitLogicalOperation(String op, Object l, Object r) throws IllegalStateException { Element rv = null; String left = (String) l; String right = (String) r; if (op == null) { - throw new Exception("Parse error, unbalanced elements. " + l.toString()); + throw new IllegalStateException("Parse error, unbalanced elements. " + l.toString()); } System.out.println(indent + op); indent += "\t"; @@ -96,10 +96,10 @@ public Element emitLogicalOperation(String op, Object l, Object r) throws Except } @Override - public Element emitUnaryOperation(String op, Element current) throws Exception { + public Element emitUnaryOperation(String op, Element current) throws IllegalStateException { Element rv = null; if (op == null) { - throw new Exception("Parse error, unknown operation: " + op + " expression:" + current); + throw new IllegalStateException("Parse error, unknown operation: " + op + " expression:" + current); } System.out.println(indent + op); indent += "\t"; diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/XmlWalker.java b/src/main/java/com/teragrep/pth_06/planner/walker/XmlWalker.java index 7b13ff1d..9d6dc2bb 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/XmlWalker.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/XmlWalker.java @@ -55,9 +55,12 @@ import org.w3c.dom.traversal.NodeFilter; import org.w3c.dom.traversal.TreeWalker; import org.xml.sax.InputSource; +import org.xml.sax.SAXException; import javax.xml.parsers.DocumentBuilder; import javax.xml.parsers.DocumentBuilderFactory; +import javax.xml.parsers.ParserConfigurationException; +import java.io.IOException; import java.io.StringReader; import java.util.Arrays; import java.util.List; @@ -81,7 +84,7 @@ public XmlWalker() { } - public T fromString(String inXml) throws Exception { + public T fromString(String inXml) throws ParserConfigurationException, IOException, SAXException { T rv; DocumentBuilderFactory factory = DocumentBuilderFactory.newInstance(); DocumentBuilder loader = factory.newDocumentBuilder(); @@ -102,9 +105,8 @@ public T fromString(String inXml) throws Exception { * @param walker * @param op operation String * @return Class which expr-part contains actual catalyst query tree - * @throws Exception */ - public T traverse(TreeWalker walker, String op) throws Exception { + public T traverse(TreeWalker walker, String op) { Node parend = walker.getCurrentNode(); Element current = ((Element) parend); T rv = null; @@ -122,7 +124,7 @@ public T traverse(TreeWalker walker, String op) throws Exception { NodeList children = parend.getChildNodes(); int count = children.getLength(); if (count < 1 || count > 2) { - throw new Exception("Error, wrong number of children:" + count + " op:" + op); + throw new IllegalStateException("Error, wrong number of children:" + count + " op:" + op); } // get left and right Node left = walker.firstChild(); @@ -150,7 +152,7 @@ public T traverse(TreeWalker walker, String op) throws Exception { } if (lft != null && rht != null) { if (op == null) { - throw new Exception("Parse error, unbalanced elements. " + lft.toString()); + throw new IllegalStateException("Parse error, unbalanced elements. " + lft.toString()); } rv = emitLogicalOperation(op, lft, rht); } @@ -181,7 +183,7 @@ else if (rht == null) { // System.out.println("XmlWalker.traverse type:"+rv.getClass().getName() +" returns:"+((T)rv).toString()); return (T) rv; } - catch (Exception e) { + catch (IllegalStateException e) { LOGGER.error(e.toString()); } return null; @@ -200,7 +202,7 @@ else if (rht == null) { * * @return correct query according to implementation class */ - abstract T emitLogicalOperation(String op, Object left, Object right) throws Exception; + abstract T emitLogicalOperation(String op, Object left, Object right) throws IllegalStateException; /** * Abstract method which is called during traverse. Emits appropriate unary operation @@ -208,7 +210,7 @@ else if (rht == null) { * @param current DOM-element * @return correct query according to implementation class */ - abstract T emitUnaryOperation(String op, Element current) throws Exception; + abstract T emitUnaryOperation(String op, Element current) throws IllegalStateException; // escape special chars inside value /** From efafc2e2cba80f4627ea54ab1ce37a5e4f071d8e Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Tue, 5 Nov 2024 15:10:37 +0200 Subject: [PATCH 57/62] Checkstyle plugin implementation complete. Implemented serializable Map conversion to HdfsOffset and KafkaOffset, solving code duplication of the conversion. Implemented additional OffsetInterface.java for HdfsOffset and KafkaOffset. --- .../pth_06/ArchiveMicroStreamReader.java | 49 ++++----------- .../pth_06/planner/HdfsQueryProcessor.java | 24 +++----- .../pth_06/planner/offset/HdfsOffset.java | 26 ++++++-- .../pth_06/planner/offset/KafkaOffset.java | 26 ++++++-- .../planner/offset/OffsetInterface.java | 60 +++++++++++++++++++ .../pth_06/planner/DatasourceOffsetTest.java | 21 ++----- 6 files changed, 127 insertions(+), 79 deletions(-) create mode 100644 src/main/java/com/teragrep/pth_06/planner/offset/OffsetInterface.java diff --git a/src/main/java/com/teragrep/pth_06/ArchiveMicroStreamReader.java b/src/main/java/com/teragrep/pth_06/ArchiveMicroStreamReader.java index 75c23940..6dba2ae0 100644 --- a/src/main/java/com/teragrep/pth_06/ArchiveMicroStreamReader.java +++ b/src/main/java/com/teragrep/pth_06/ArchiveMicroStreamReader.java @@ -57,17 +57,14 @@ import com.teragrep.pth_06.task.HdfsMicroBatchInputPartition; import com.teragrep.pth_06.task.TeragrepPartitionReaderFactory; import com.teragrep.pth_06.task.KafkaMicroBatchInputPartition; -import org.apache.kafka.common.TopicPartition; import org.apache.spark.sql.connector.read.InputPartition; import org.apache.spark.sql.connector.read.PartitionReaderFactory; import org.apache.spark.sql.connector.read.streaming.MicroBatchStream; import org.apache.spark.sql.connector.read.streaming.Offset; import org.apache.spark.sql.execution.streaming.LongOffset; -import java.util.HashMap; import java.util.LinkedList; import java.util.List; -import java.util.Map; // logger @@ -173,12 +170,8 @@ public Offset initialOffset() { KafkaOffset kafkaOffset = new KafkaOffset(); // stub if (this.config.isHdfsEnabled) { - Map offset = this.hq.getBeginningOffsets().getOffsetMap(); - Map serializedHdfsOffset = new HashMap<>(offset.size()); - for (Map.Entry entry : offset.entrySet()) { - serializedHdfsOffset.put(entry.getKey().toString(), entry.getValue()); // offset - } - hdfsOffset = new HdfsOffset(serializedHdfsOffset); + hdfsOffset = new HdfsOffset(this.hq.getBeginningOffsets().getOffsetMap()); + hdfsOffset.serialize(); } if (this.config.isArchiveEnabled) { longOffset = new LongOffset(this.aq.getInitialOffset() - 3600L); @@ -186,28 +179,17 @@ public Offset initialOffset() { if (this.config.isKafkaEnabled) { if (this.config.isHdfsEnabled) { if (hdfsOffsets.size() > 0) { - Map serializedKafkaOffset = new HashMap<>( - this.kq.getConsumerPositions(hdfsOffsets).size() - ); - for (Map.Entry entry : this.kq.getConsumerPositions(hdfsOffsets).entrySet()) { - serializedKafkaOffset.put(entry.getKey().toString(), entry.getValue()); // offset - } - kafkaOffset = new KafkaOffset(serializedKafkaOffset); + kafkaOffset = new KafkaOffset(this.kq.getConsumerPositions(hdfsOffsets)); + kafkaOffset.serialize(); } else { - Map serializedKafkaOffset = new HashMap<>(this.kq.getBeginningOffsets(null).size()); - for (Map.Entry entry : this.kq.getBeginningOffsets(null).entrySet()) { - serializedKafkaOffset.put(entry.getKey().toString(), entry.getValue()); // offset - } - kafkaOffset = new KafkaOffset(serializedKafkaOffset); + kafkaOffset = new KafkaOffset(this.kq.getBeginningOffsets(null)); + kafkaOffset.serialize(); } } else { - Map serializedKafkaOffset = new HashMap<>(this.kq.getBeginningOffsets(null).size()); - for (Map.Entry entry : this.kq.getBeginningOffsets(null).entrySet()) { - serializedKafkaOffset.put(entry.getKey().toString(), entry.getValue()); // offset - } - kafkaOffset = new KafkaOffset(serializedKafkaOffset); + kafkaOffset = new KafkaOffset(this.kq.getBeginningOffsets(null)); + kafkaOffset.serialize(); } } if (hdfsOffset.isStub() && longOffset == null && kafkaOffset.isStub()) { @@ -254,22 +236,15 @@ public Offset latestOffset() { KafkaOffset kafkaOffset = new KafkaOffset(); if (this.config.isHdfsEnabled) { - Map offset = this.hq.incrementAndGetLatestOffset().getOffsetMap(); - Map serializedHdfsOffset = new HashMap<>(offset.size()); - for (Map.Entry entry : offset.entrySet()) { - serializedHdfsOffset.put(entry.getKey().toString(), entry.getValue()); // offset - } - hdfsOffset = new HdfsOffset(serializedHdfsOffset); + hdfsOffset = new HdfsOffset(this.hq.incrementAndGetLatestOffset().getOffsetMap()); + hdfsOffset.serialize(); } if (this.config.isArchiveEnabled) { longOffset = new LongOffset(this.aq.incrementAndGetLatestOffset()); } if (this.config.isKafkaEnabled) { - Map serializedKafkaOffset = new HashMap<>(this.kq.getInitialEndOffsets().size()); - for (Map.Entry entry : this.kq.getInitialEndOffsets().entrySet()) { - serializedKafkaOffset.put(entry.getKey().toString(), entry.getValue()); // offset - } - kafkaOffset = new KafkaOffset(serializedKafkaOffset); + kafkaOffset = new KafkaOffset(this.kq.getInitialEndOffsets()); + kafkaOffset.serialize(); } if (hdfsOffset.isStub() && longOffset == null && kafkaOffset.isStub()) { throw new IllegalStateException("no datasources enabled, can't get latest offset"); diff --git a/src/main/java/com/teragrep/pth_06/planner/HdfsQueryProcessor.java b/src/main/java/com/teragrep/pth_06/planner/HdfsQueryProcessor.java index 1f1bdd53..28d81b3d 100644 --- a/src/main/java/com/teragrep/pth_06/planner/HdfsQueryProcessor.java +++ b/src/main/java/com/teragrep/pth_06/planner/HdfsQueryProcessor.java @@ -216,11 +216,9 @@ public HdfsOffset getBeginningOffsets() { } } } - Map serializedHdfsOffset = new HashMap<>(startOffset.size()); - for (Map.Entry entry : startOffset.entrySet()) { - serializedHdfsOffset.put(entry.getKey().toString(), entry.getValue()); // offset - } - return new HdfsOffset(serializedHdfsOffset); + HdfsOffset hdfsOffset = new HdfsOffset(startOffset); + hdfsOffset.serialize(); + return hdfsOffset; } // returns the end offsets for all available (aka. filtered) topic partitions. @@ -241,11 +239,9 @@ public HdfsOffset getInitialEndOffsets() { } } } - Map serializedHdfsOffset = new HashMap<>(endOffset.size()); - for (Map.Entry entry : endOffset.entrySet()) { - serializedHdfsOffset.put(entry.getKey().toString(), entry.getValue()); // offset - } - return new HdfsOffset(serializedHdfsOffset); + HdfsOffset hdfsOffset = new HdfsOffset(endOffset); + hdfsOffset.serialize(); + return hdfsOffset; } // Increments the latest offset values and returns that incremented offsets. Works by pulling data from the topicPartitionList until weight limit is reached. @@ -267,11 +263,9 @@ public HdfsOffset incrementAndGetLatestOffset() { batchSizeLimit.add(r.hdfsFileSize); } } - Map serializedHdfsOffset = new HashMap<>(latestHdfsOffsetMap.size()); - for (Map.Entry entry : latestHdfsOffsetMap.entrySet()) { - serializedHdfsOffset.put(entry.getKey().toString(), entry.getValue()); // offset - } - return new HdfsOffset(serializedHdfsOffset); + HdfsOffset hdfsOffset = new HdfsOffset(latestHdfsOffsetMap); + hdfsOffset.serialize(); + return hdfsOffset; } @Override diff --git a/src/main/java/com/teragrep/pth_06/planner/offset/HdfsOffset.java b/src/main/java/com/teragrep/pth_06/planner/offset/HdfsOffset.java index e9442d53..9bab665f 100644 --- a/src/main/java/com/teragrep/pth_06/planner/offset/HdfsOffset.java +++ b/src/main/java/com/teragrep/pth_06/planner/offset/HdfsOffset.java @@ -57,29 +57,32 @@ // Class for representing a serializable offset of HDFS data source. // S3 has epoch hours as offsets, kafka has native TopicPartition offsets and HDFS should have file-metadata (use same format as in Kafka, topicpartition + record offset, which can be extracted from the metadata). -public class HdfsOffset extends Offset implements Serializable { +public class HdfsOffset extends Offset implements Serializable, OffsetInterface { private final Map serializedHdfsOffset; private final boolean stub; + private final transient Map hdfsOffset; public HdfsOffset() { - this(new HashMap<>(), true); + this(new HashMap<>(), new HashMap<>(), true); } public HdfsOffset(String s) { - this(new Gson().fromJson(s, new TypeToken>() { + this(new HashMap<>(), new Gson().fromJson(s, new TypeToken>() { }.getType()), false); } - public HdfsOffset(Map serializedHdfsOffset) { - this(serializedHdfsOffset, false); + public HdfsOffset(Map serializedHdfsOffset) { + this(serializedHdfsOffset, new HashMap<>(), false); } - public HdfsOffset(Map serializedHdfsOffset, boolean stub) { + public HdfsOffset(Map hdfsOffset, Map serializedHdfsOffset, boolean stub) { + this.hdfsOffset = hdfsOffset; this.serializedHdfsOffset = serializedHdfsOffset; this.stub = stub; } + @Override public Map getOffsetMap() { Map rv = new HashMap<>(serializedHdfsOffset.size()); @@ -96,10 +99,21 @@ public Map getOffsetMap() { return rv; } + @Override public boolean isStub() { return stub; } + @Override + public void serialize() { + if (!hdfsOffset.isEmpty()) { + for (Map.Entry entry : hdfsOffset.entrySet()) { + serializedHdfsOffset.put(entry.getKey().toString(), entry.getValue()); + } + hdfsOffset.clear(); + } + } + @Override public String json() { Gson gson = new Gson(); diff --git a/src/main/java/com/teragrep/pth_06/planner/offset/KafkaOffset.java b/src/main/java/com/teragrep/pth_06/planner/offset/KafkaOffset.java index b9ed8596..48d30637 100644 --- a/src/main/java/com/teragrep/pth_06/planner/offset/KafkaOffset.java +++ b/src/main/java/com/teragrep/pth_06/planner/offset/KafkaOffset.java @@ -60,29 +60,32 @@ * @since 08/06/2022 * @author Mikko Kortelainen */ -public class KafkaOffset extends Offset implements Serializable { +public class KafkaOffset extends Offset implements Serializable, OffsetInterface { private final Map serializedKafkaOffset; private final boolean stub; + private final transient Map kafkaOffset; public KafkaOffset() { - this(new HashMap<>(), true); + this(new HashMap<>(), new HashMap<>(), true); } public KafkaOffset(String s) { - this(new Gson().fromJson(s, new TypeToken>() { + this(new HashMap<>(), new Gson().fromJson(s, new TypeToken>() { }.getType()), false); } - public KafkaOffset(Map serializedKafkaOffset) { - this(serializedKafkaOffset, false); + public KafkaOffset(Map kafkaOffset) { + this(kafkaOffset, new HashMap<>(), false); } - public KafkaOffset(Map serializedKafkaOffset, boolean stub) { + public KafkaOffset(Map kafkaOffset, Map serializedKafkaOffset, boolean stub) { + this.kafkaOffset = kafkaOffset; this.serializedKafkaOffset = serializedKafkaOffset; this.stub = stub; } + @Override public Map getOffsetMap() { Map rv = new HashMap<>(serializedKafkaOffset.size()); @@ -99,10 +102,21 @@ public Map getOffsetMap() { return rv; } + @Override public boolean isStub() { return stub; } + @Override + public void serialize() { + if (!kafkaOffset.isEmpty()) { + for (Map.Entry entry : kafkaOffset.entrySet()) { + serializedKafkaOffset.put(entry.getKey().toString(), entry.getValue()); + } + kafkaOffset.clear(); + } + } + @Override public String json() { Gson gson = new Gson(); diff --git a/src/main/java/com/teragrep/pth_06/planner/offset/OffsetInterface.java b/src/main/java/com/teragrep/pth_06/planner/offset/OffsetInterface.java new file mode 100644 index 00000000..49039f7b --- /dev/null +++ b/src/main/java/com/teragrep/pth_06/planner/offset/OffsetInterface.java @@ -0,0 +1,60 @@ +/* + * Teragrep Archive Datasource (pth_06) + * 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.pth_06.planner.offset; + +import org.apache.kafka.common.TopicPartition; + +import java.util.Map; + +public interface OffsetInterface { + + public abstract Map getOffsetMap(); + + public abstract boolean isStub(); + + public abstract void serialize(); + +} diff --git a/src/test/java/com/teragrep/pth_06/planner/DatasourceOffsetTest.java b/src/test/java/com/teragrep/pth_06/planner/DatasourceOffsetTest.java index 01c289be..1d6625b8 100644 --- a/src/test/java/com/teragrep/pth_06/planner/DatasourceOffsetTest.java +++ b/src/test/java/com/teragrep/pth_06/planner/DatasourceOffsetTest.java @@ -63,11 +63,8 @@ public void serdeTest() { LongOffset longOffset = new LongOffset(0L); Map topicPartitionLongMap = new HashMap<>(); topicPartitionLongMap.put(new TopicPartition("test", 0), 0L); - Map serializedKafkaOffset = new HashMap<>(topicPartitionLongMap.size()); - for (Map.Entry entry : topicPartitionLongMap.entrySet()) { - serializedKafkaOffset.put(entry.getKey().toString(), entry.getValue()); // offset - } - KafkaOffset kafkaOffset = new KafkaOffset(serializedKafkaOffset); + KafkaOffset kafkaOffset = new KafkaOffset(topicPartitionLongMap); + kafkaOffset.serialize(); DatasourceOffset datasourceOffset = new DatasourceOffset(longOffset, kafkaOffset); String ser = datasourceOffset.json(); @@ -81,11 +78,8 @@ public void serdeTest() { public void kafkaOffsetSerdeTest() { Map topicPartitionLongMap = new HashMap<>(); topicPartitionLongMap.put(new TopicPartition("test", 777), 9999L); - Map serializedKafkaOffset = new HashMap<>(topicPartitionLongMap.size()); - for (Map.Entry entry : topicPartitionLongMap.entrySet()) { - serializedKafkaOffset.put(entry.getKey().toString(), entry.getValue()); // offset - } - KafkaOffset kafkaOffset = new KafkaOffset(serializedKafkaOffset); + KafkaOffset kafkaOffset = new KafkaOffset(topicPartitionLongMap); + kafkaOffset.serialize(); String ser = kafkaOffset.json(); KafkaOffset deser = new KafkaOffset(ser); @@ -104,11 +98,8 @@ public void kafkaOffsetSerdeTest() { public void HdfsOffsetSerdeTest() { Map topicPartitionLongMap = new HashMap<>(); topicPartitionLongMap.put(new TopicPartition("test", 777), 9999L); - Map serializedHdfsOffset = new HashMap<>(topicPartitionLongMap.size()); - for (Map.Entry entry : topicPartitionLongMap.entrySet()) { - serializedHdfsOffset.put(entry.getKey().toString(), entry.getValue()); - } - HdfsOffset hdfsOffset = new HdfsOffset(serializedHdfsOffset); + HdfsOffset hdfsOffset = new HdfsOffset(topicPartitionLongMap); + hdfsOffset.serialize(); String ser = hdfsOffset.json(); HdfsOffset deser = new HdfsOffset(ser); From 932c39bcafe6b587acb67811c4f93590bbc0cb30 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Thu, 7 Nov 2024 16:02:40 +0200 Subject: [PATCH 58/62] Refactored hashCode() methods to use Objects.hash(). --- .../teragrep/pth_06/config/ConditionConfig.java | 11 +++-------- .../walker/conditions/EarliestCondition.java | 7 ++----- .../walker/conditions/ElementCondition.java | 17 +++-------------- .../walker/conditions/HostCondition.java | 11 +++-------- .../walker/conditions/IndexCondition.java | 12 +++--------- .../conditions/IndexStatementCondition.java | 14 ++------------ .../walker/conditions/LatestCondition.java | 7 ++----- .../walker/conditions/SourceTypeCondition.java | 12 +++--------- 8 files changed, 21 insertions(+), 70 deletions(-) diff --git a/src/main/java/com/teragrep/pth_06/config/ConditionConfig.java b/src/main/java/com/teragrep/pth_06/config/ConditionConfig.java index 20ddc77d..b81d1f17 100644 --- a/src/main/java/com/teragrep/pth_06/config/ConditionConfig.java +++ b/src/main/java/com/teragrep/pth_06/config/ConditionConfig.java @@ -47,6 +47,8 @@ import org.jooq.DSLContext; +import java.util.Objects; + public final class ConditionConfig { private final DSLContext ctx; @@ -123,13 +125,6 @@ public boolean equals(Object object) { @Override public int hashCode() { - int result = 17; - result = 31 * result + (bloomEnabled ? 1 : 0); - result = 31 * result + (streamQuery ? 1 : 0); - result = 31 * result + (withoutFilters ? 1 : 0); - if (ctx != null) { - result = 31 * result + ctx.hashCode(); - } - return result; + return Objects.hash(ctx, streamQuery, bloomEnabled, withoutFilters); } } diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/EarliestCondition.java b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/EarliestCondition.java index 30319afc..83a77cd6 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/EarliestCondition.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/EarliestCondition.java @@ -49,6 +49,7 @@ import java.sql.Date; import java.time.Instant; +import java.util.Objects; import static com.teragrep.pth_06.jooq.generated.journaldb.Journaldb.JOURNALDB; @@ -102,10 +103,6 @@ public boolean equals(final Object object) { @Override public int hashCode() { - int result = 17; - if (value != null) { - result = 31 * result + value.hashCode(); - } - return result; + return Objects.hashCode(value); } } diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/ElementCondition.java b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/ElementCondition.java index d4e51a25..276e10db 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/ElementCondition.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/ElementCondition.java @@ -55,6 +55,8 @@ import java.util.Set; +import java.util.Objects; + /** * Creates a query condition from provided dom element */ @@ -149,19 +151,6 @@ public boolean equals(final Object object) { @Override public int hashCode() { - int result = 17; - if (element.getTagName() != null) { - result = 31 * result + element.getTagName().hashCode(); - } - if (element.getAttribute("operation") != null) { - result = 31 * result + element.getAttribute("operation").hashCode(); - } - if (element.getAttribute("value") != null) { - result = 31 * result + element.getAttribute("value").hashCode(); - } - if (config != null) { - result = 31 * result + config.hashCode(); - } - return result; + return Objects.hash(element, config); } } diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/HostCondition.java b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/HostCondition.java index d2632405..eaa1cf59 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/HostCondition.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/HostCondition.java @@ -48,6 +48,8 @@ import com.teragrep.pth_06.planner.StreamDBClient; import org.jooq.Condition; +import java.util.Objects; + import static com.teragrep.pth_06.jooq.generated.streamdb.Streamdb.STREAMDB; public final class HostCondition implements QueryCondition { @@ -94,13 +96,6 @@ public boolean equals(final Object object) { @Override public int hashCode() { - int result = 17; - if (value != null) { - result = 31 * result + value.hashCode(); - } - if (operation != null) { - result = 31 * result + operation.hashCode(); - } - return result; + return Objects.hash(value, operation, streamQuery); } } diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/IndexCondition.java b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/IndexCondition.java index 397d3322..2cb3d0e0 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/IndexCondition.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/IndexCondition.java @@ -48,6 +48,8 @@ import com.teragrep.pth_06.planner.StreamDBClient; import org.jooq.Condition; +import java.util.Objects; + import static com.teragrep.pth_06.jooq.generated.streamdb.Streamdb.STREAMDB; public final class IndexCondition implements QueryCondition { @@ -95,14 +97,6 @@ public boolean equals(final Object object) { @Override public int hashCode() { - int result = 17; - if (value != null) { - result = 31 * result + value.hashCode(); - } - if (operation != null) { - result = 31 * result + operation.hashCode(); - } - result = 31 * result + (streamQuery ? 1 : 0); - return result; + return Objects.hash(value, operation, streamQuery); } } diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/IndexStatementCondition.java b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/IndexStatementCondition.java index f86838f8..4adcdb08 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/IndexStatementCondition.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/IndexStatementCondition.java @@ -54,6 +54,7 @@ import org.slf4j.LoggerFactory; import java.util.HashSet; +import java.util.Objects; import java.util.Set; import static com.teragrep.pth_06.jooq.generated.bloomdb.Bloomdb.BLOOMDB; @@ -155,17 +156,6 @@ public boolean equals(final Object object) { @Override public int hashCode() { - int result = 17; - if (value != null) { - result = 31 * result + value.hashCode(); - } - if (tokenizer != null) { - result = 31 * result + tokenizer.hashCode(); - } - if (config != null) { - result = 31 * result + config.hashCode(); - } - return result; + return Objects.hash(value, config, tokenizer); } - } diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/LatestCondition.java b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/LatestCondition.java index 7fa16188..e26add38 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/LatestCondition.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/LatestCondition.java @@ -49,6 +49,7 @@ import java.sql.Date; import java.time.Instant; +import java.util.Objects; import static com.teragrep.pth_06.jooq.generated.journaldb.Journaldb.JOURNALDB; @@ -99,10 +100,6 @@ public boolean equals(final Object object) { @Override public int hashCode() { - int result = 17; - if (value != null) { - result = 31 * result + value.hashCode(); - } - return result; + return Objects.hashCode(value); } } diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/SourceTypeCondition.java b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/SourceTypeCondition.java index dc58be82..6ee21fff 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/SourceTypeCondition.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/SourceTypeCondition.java @@ -48,6 +48,8 @@ import com.teragrep.pth_06.planner.StreamDBClient; import org.jooq.Condition; +import java.util.Objects; + import static com.teragrep.pth_06.jooq.generated.streamdb.Streamdb.STREAMDB; public final class SourceTypeCondition implements QueryCondition { @@ -94,14 +96,6 @@ public boolean equals(final Object object) { @Override public int hashCode() { - int result = 17; - if (value != null) { - result = 31 * result + value.hashCode(); - } - if (operation != null) { - result = 31 * result + operation.hashCode(); - } - result = 31 * result + (streamQuery ? 1 : 0); - return result; + return Objects.hash(value, operation, streamQuery); } } From 45c4a708951f82653a2ee9f881f63e398f2a1b8f Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Fri, 8 Nov 2024 09:32:02 +0200 Subject: [PATCH 59/62] Renamed OffsetInterface to Offset. Spotless. --- .../java/com/teragrep/pth_06/planner/offset/HdfsOffset.java | 3 +-- .../java/com/teragrep/pth_06/planner/offset/KafkaOffset.java | 3 +-- .../planner/offset/{OffsetInterface.java => Offset.java} | 2 +- 3 files changed, 3 insertions(+), 5 deletions(-) rename src/main/java/com/teragrep/pth_06/planner/offset/{OffsetInterface.java => Offset.java} (98%) diff --git a/src/main/java/com/teragrep/pth_06/planner/offset/HdfsOffset.java b/src/main/java/com/teragrep/pth_06/planner/offset/HdfsOffset.java index 9bab665f..52d1dd55 100644 --- a/src/main/java/com/teragrep/pth_06/planner/offset/HdfsOffset.java +++ b/src/main/java/com/teragrep/pth_06/planner/offset/HdfsOffset.java @@ -48,7 +48,6 @@ import com.google.common.reflect.TypeToken; import com.google.gson.Gson; import org.apache.kafka.common.TopicPartition; -import org.apache.spark.sql.connector.read.streaming.Offset; import java.io.Serializable; import java.util.HashMap; @@ -57,7 +56,7 @@ // Class for representing a serializable offset of HDFS data source. // S3 has epoch hours as offsets, kafka has native TopicPartition offsets and HDFS should have file-metadata (use same format as in Kafka, topicpartition + record offset, which can be extracted from the metadata). -public class HdfsOffset extends Offset implements Serializable, OffsetInterface { +public class HdfsOffset extends org.apache.spark.sql.connector.read.streaming.Offset implements Serializable, Offset { private final Map serializedHdfsOffset; private final boolean stub; diff --git a/src/main/java/com/teragrep/pth_06/planner/offset/KafkaOffset.java b/src/main/java/com/teragrep/pth_06/planner/offset/KafkaOffset.java index 48d30637..390c1f91 100644 --- a/src/main/java/com/teragrep/pth_06/planner/offset/KafkaOffset.java +++ b/src/main/java/com/teragrep/pth_06/planner/offset/KafkaOffset.java @@ -48,7 +48,6 @@ import com.google.common.reflect.TypeToken; import com.google.gson.Gson; import org.apache.kafka.common.TopicPartition; -import org.apache.spark.sql.connector.read.streaming.Offset; import java.io.Serializable; import java.util.HashMap; @@ -60,7 +59,7 @@ * @since 08/06/2022 * @author Mikko Kortelainen */ -public class KafkaOffset extends Offset implements Serializable, OffsetInterface { +public class KafkaOffset extends org.apache.spark.sql.connector.read.streaming.Offset implements Serializable, Offset { private final Map serializedKafkaOffset; private final boolean stub; diff --git a/src/main/java/com/teragrep/pth_06/planner/offset/OffsetInterface.java b/src/main/java/com/teragrep/pth_06/planner/offset/Offset.java similarity index 98% rename from src/main/java/com/teragrep/pth_06/planner/offset/OffsetInterface.java rename to src/main/java/com/teragrep/pth_06/planner/offset/Offset.java index 49039f7b..cb9683c5 100644 --- a/src/main/java/com/teragrep/pth_06/planner/offset/OffsetInterface.java +++ b/src/main/java/com/teragrep/pth_06/planner/offset/Offset.java @@ -49,7 +49,7 @@ import java.util.Map; -public interface OffsetInterface { +public interface Offset { public abstract Map getOffsetMap(); From abe5823b0320cf06d5ef9e1e70721716d1bba04d Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Fri, 8 Nov 2024 14:48:26 +0200 Subject: [PATCH 60/62] Replaced RuntimeException with more descriptive UncheckedIOException. --- .../java/com/teragrep/pth_06/planner/HdfsQueryProcessor.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/main/java/com/teragrep/pth_06/planner/HdfsQueryProcessor.java b/src/main/java/com/teragrep/pth_06/planner/HdfsQueryProcessor.java index 28d81b3d..d194fbb0 100644 --- a/src/main/java/com/teragrep/pth_06/planner/HdfsQueryProcessor.java +++ b/src/main/java/com/teragrep/pth_06/planner/HdfsQueryProcessor.java @@ -58,6 +58,7 @@ import javax.xml.parsers.ParserConfigurationException; import java.io.IOException; +import java.io.UncheckedIOException; import java.util.HashMap; import java.util.Iterator; import java.util.LinkedList; @@ -114,7 +115,7 @@ public HdfsQueryProcessor(Config config) { ); } catch (IOException e) { - throw new RuntimeException(e); + throw new UncheckedIOException(e); } // Generate the new starting partition offsets for kafka. From d83af235ed56eea6e5c60fdbd75c53fcd8361917 Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Fri, 8 Nov 2024 16:24:10 +0200 Subject: [PATCH 61/62] Rebase. Fixed leftover conflicts from rebase. --- src/main/java/com/teragrep/pth_06/planner/StreamDBClient.java | 1 + .../planner/walker/conditions/IndexStatementCondition.java | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/src/main/java/com/teragrep/pth_06/planner/StreamDBClient.java b/src/main/java/com/teragrep/pth_06/planner/StreamDBClient.java index e28b6829..d6c46adb 100644 --- a/src/main/java/com/teragrep/pth_06/planner/StreamDBClient.java +++ b/src/main/java/com/teragrep/pth_06/planner/StreamDBClient.java @@ -45,6 +45,7 @@ */ package com.teragrep.pth_06.planner; +import java.io.IOException; import java.sql.*; import java.time.Instant; import java.util.Set; diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/IndexStatementCondition.java b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/IndexStatementCondition.java index 4adcdb08..8fa4efc5 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/IndexStatementCondition.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/IndexStatementCondition.java @@ -156,6 +156,6 @@ public boolean equals(final Object object) { @Override public int hashCode() { - return Objects.hash(value, config, tokenizer); + return Objects.hash(value, config); } } From d7550fac59558ab4dec0d416ecbdbd4700b21ffe Mon Sep 17 00:00:00 2001 From: Tiihott <48@teragrep.com> Date: Mon, 11 Nov 2024 10:20:38 +0200 Subject: [PATCH 62/62] Rebase Checkstyle code cleanup. --- .../planner/bloomfilter/CategoryTable.java | 2 +- .../planner/bloomfilter/CategoryTableImpl.java | 9 ++++++--- .../bloomfilter/ConditionMatchBloomDBTables.java | 9 ++++++--- .../planner/bloomfilter/DatabaseTables.java | 2 +- .../planner/bloomfilter/RegexExtractedValue.java | 6 ++++-- .../bloomfilter/SearchTermBloomFilter.java | 6 ++++-- .../TableFilterTypesFromMetadata.java | 6 ++++-- .../pth_06/planner/bloomfilter/TableFilters.java | 6 ++++-- .../pth_06/planner/bloomfilter/TableRecords.java | 2 +- .../pth_06/planner/bloomfilter/Tokenizable.java | 2 +- .../planner/bloomfilter/TokenizedValue.java | 6 ++++-- .../planner/bloomfilter/TokensAsStrings.java | 6 ++++-- .../walker/conditions/BloomQueryCondition.java | 4 ++-- .../conditions/CategoryTableCondition.java | 6 ++++-- .../walker/conditions/RegexLikeCondition.java | 6 ++++-- .../planner/walker/conditions/ValidElement.java | 16 +++++++++++++--- 16 files changed, 63 insertions(+), 31 deletions(-) diff --git a/src/main/java/com/teragrep/pth_06/planner/bloomfilter/CategoryTable.java b/src/main/java/com/teragrep/pth_06/planner/bloomfilter/CategoryTable.java index 03f7499a..ca622d46 100644 --- a/src/main/java/com/teragrep/pth_06/planner/bloomfilter/CategoryTable.java +++ b/src/main/java/com/teragrep/pth_06/planner/bloomfilter/CategoryTable.java @@ -47,6 +47,6 @@ public interface CategoryTable { - void create(); + public abstract void create(); } diff --git a/src/main/java/com/teragrep/pth_06/planner/bloomfilter/CategoryTableImpl.java b/src/main/java/com/teragrep/pth_06/planner/bloomfilter/CategoryTableImpl.java index d13c2678..63f908aa 100644 --- a/src/main/java/com/teragrep/pth_06/planner/bloomfilter/CategoryTableImpl.java +++ b/src/main/java/com/teragrep/pth_06/planner/bloomfilter/CategoryTableImpl.java @@ -138,12 +138,15 @@ public void create() { */ @Override public boolean equals(final Object object) { - if (this == object) + if (this == object) { return true; - if (object == null) + } + if (object == null) { return false; - if (object.getClass() != this.getClass()) + } + if (object.getClass() != this.getClass()) { return false; + } final CategoryTableImpl cast = (CategoryTableImpl) object; return this.originTable.equals(cast.originTable) && this.ctx == cast.ctx && // equal only if same instance of DSLContext this.bloomTermId == cast.bloomTermId && this.tableFilters.equals(cast.tableFilters); diff --git a/src/main/java/com/teragrep/pth_06/planner/bloomfilter/ConditionMatchBloomDBTables.java b/src/main/java/com/teragrep/pth_06/planner/bloomfilter/ConditionMatchBloomDBTables.java index 3f32b31c..16ddb7e8 100644 --- a/src/main/java/com/teragrep/pth_06/planner/bloomfilter/ConditionMatchBloomDBTables.java +++ b/src/main/java/com/teragrep/pth_06/planner/bloomfilter/ConditionMatchBloomDBTables.java @@ -111,12 +111,15 @@ public List> tables() { */ @Override public boolean equals(final Object object) { - if (this == object) + if (this == object) { return true; - if (object == null) + } + if (object == null) { return false; - if (object.getClass() != this.getClass()) + } + if (object.getClass() != this.getClass()) { return false; + } final ConditionMatchBloomDBTables cast = (ConditionMatchBloomDBTables) object; return this.condition.equals(cast.condition) && this.ctx == cast.ctx; } diff --git a/src/main/java/com/teragrep/pth_06/planner/bloomfilter/DatabaseTables.java b/src/main/java/com/teragrep/pth_06/planner/bloomfilter/DatabaseTables.java index 2a4869f6..f0511676 100644 --- a/src/main/java/com/teragrep/pth_06/planner/bloomfilter/DatabaseTables.java +++ b/src/main/java/com/teragrep/pth_06/planner/bloomfilter/DatabaseTables.java @@ -51,5 +51,5 @@ public interface DatabaseTables { - List> tables(); + public abstract List> tables(); } diff --git a/src/main/java/com/teragrep/pth_06/planner/bloomfilter/RegexExtractedValue.java b/src/main/java/com/teragrep/pth_06/planner/bloomfilter/RegexExtractedValue.java index a723a0dc..a3fe4b0e 100644 --- a/src/main/java/com/teragrep/pth_06/planner/bloomfilter/RegexExtractedValue.java +++ b/src/main/java/com/teragrep/pth_06/planner/bloomfilter/RegexExtractedValue.java @@ -75,10 +75,12 @@ public List tokens() { @Override public boolean equals(final Object object) { - if (this == object) + if (this == object) { return true; - if (object == null || getClass() != object.getClass()) + } + if (object == null || getClass() != object.getClass()) { return false; + } final RegexExtractedValue cast = (RegexExtractedValue) object; return value.equals(cast.value) && pattern.equals(cast.pattern); } diff --git a/src/main/java/com/teragrep/pth_06/planner/bloomfilter/SearchTermBloomFilter.java b/src/main/java/com/teragrep/pth_06/planner/bloomfilter/SearchTermBloomFilter.java index a30dc70f..c67bbd0c 100644 --- a/src/main/java/com/teragrep/pth_06/planner/bloomfilter/SearchTermBloomFilter.java +++ b/src/main/java/com/teragrep/pth_06/planner/bloomfilter/SearchTermBloomFilter.java @@ -99,10 +99,12 @@ public byte[] bytes() { @Override public boolean equals(final Object object) { - if (this == object) + if (this == object) { return true; - if (object == null || getClass() != object.getClass()) + } + if (object == null || getClass() != object.getClass()) { return false; + } final SearchTermBloomFilter cast = (SearchTermBloomFilter) object; return expected.equals(cast.expected) && fpp.equals(cast.fpp) && stringTokens.equals(cast.stringTokens); } diff --git a/src/main/java/com/teragrep/pth_06/planner/bloomfilter/TableFilterTypesFromMetadata.java b/src/main/java/com/teragrep/pth_06/planner/bloomfilter/TableFilterTypesFromMetadata.java index a5efa749..4a3111aa 100644 --- a/src/main/java/com/teragrep/pth_06/planner/bloomfilter/TableFilterTypesFromMetadata.java +++ b/src/main/java/com/teragrep/pth_06/planner/bloomfilter/TableFilterTypesFromMetadata.java @@ -120,10 +120,12 @@ public Result toResult() { */ @Override public boolean equals(final Object object) { - if (this == object) + if (this == object) { return true; - if (object == null || getClass() != object.getClass()) + } + if (object == null || getClass() != object.getClass()) { return false; + } final TableFilterTypesFromMetadata cast = (TableFilterTypesFromMetadata) object; return ctx == cast.ctx && Objects.equals(table, cast.table) && Objects .equals(expectedField, cast.expectedField) && Objects.equals(fppField, cast.fppField) && Objects diff --git a/src/main/java/com/teragrep/pth_06/planner/bloomfilter/TableFilters.java b/src/main/java/com/teragrep/pth_06/planner/bloomfilter/TableFilters.java index 54184424..2f4c52d3 100644 --- a/src/main/java/com/teragrep/pth_06/planner/bloomfilter/TableFilters.java +++ b/src/main/java/com/teragrep/pth_06/planner/bloomfilter/TableFilters.java @@ -139,10 +139,12 @@ public Batch asBatch() { */ @Override public boolean equals(final Object object) { - if (this == object) + if (this == object) { return true; - if (object == null || getClass() != object.getClass()) + } + if (object == null || getClass() != object.getClass()) { return false; + } final TableFilters cast = (TableFilters) object; return bloomTermId == cast.bloomTermId && recordsInMetadata.equals(cast.recordsInMetadata) && ctx == cast.ctx && table.equals(cast.table) && thisTable.equals(cast.thisTable) && searchTerm.equals(cast.searchTerm); diff --git a/src/main/java/com/teragrep/pth_06/planner/bloomfilter/TableRecords.java b/src/main/java/com/teragrep/pth_06/planner/bloomfilter/TableRecords.java index 932070f8..91b870d8 100644 --- a/src/main/java/com/teragrep/pth_06/planner/bloomfilter/TableRecords.java +++ b/src/main/java/com/teragrep/pth_06/planner/bloomfilter/TableRecords.java @@ -50,5 +50,5 @@ public interface TableRecords { - Result toResult(); + public abstract Result toResult(); } diff --git a/src/main/java/com/teragrep/pth_06/planner/bloomfilter/Tokenizable.java b/src/main/java/com/teragrep/pth_06/planner/bloomfilter/Tokenizable.java index e209f536..c40ce443 100644 --- a/src/main/java/com/teragrep/pth_06/planner/bloomfilter/Tokenizable.java +++ b/src/main/java/com/teragrep/pth_06/planner/bloomfilter/Tokenizable.java @@ -49,5 +49,5 @@ public interface Tokenizable { - List tokens(); + public abstract List tokens(); } diff --git a/src/main/java/com/teragrep/pth_06/planner/bloomfilter/TokenizedValue.java b/src/main/java/com/teragrep/pth_06/planner/bloomfilter/TokenizedValue.java index fe32e624..6a07b959 100644 --- a/src/main/java/com/teragrep/pth_06/planner/bloomfilter/TokenizedValue.java +++ b/src/main/java/com/teragrep/pth_06/planner/bloomfilter/TokenizedValue.java @@ -67,10 +67,12 @@ public List tokens() { @Override public boolean equals(final Object object) { - if (this == object) + if (this == object) { return true; - if (object == null || object.getClass() != this.getClass()) + } + if (object == null || object.getClass() != this.getClass()) { return false; + } final TokenizedValue cast = (TokenizedValue) object; return value.equals(cast.value); } diff --git a/src/main/java/com/teragrep/pth_06/planner/bloomfilter/TokensAsStrings.java b/src/main/java/com/teragrep/pth_06/planner/bloomfilter/TokensAsStrings.java index b0c9dce8..fa2d9985 100644 --- a/src/main/java/com/teragrep/pth_06/planner/bloomfilter/TokensAsStrings.java +++ b/src/main/java/com/teragrep/pth_06/planner/bloomfilter/TokensAsStrings.java @@ -66,10 +66,12 @@ public List tokens() { @Override public boolean equals(final Object object) { - if (this == object) + if (this == object) { return true; - if (object == null || object.getClass() != this.getClass()) + } + if (object == null || object.getClass() != this.getClass()) { return false; + } final TokensAsStrings cast = (TokensAsStrings) object; return origin.equals(cast.origin); } diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/BloomQueryCondition.java b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/BloomQueryCondition.java index a3a0a360..42d11fcb 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/BloomQueryCondition.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/BloomQueryCondition.java @@ -51,7 +51,7 @@ public interface BloomQueryCondition { - boolean isBloomSearchCondition(); + public abstract boolean isBloomSearchCondition(); - Set> requiredTables(); + public abstract Set> requiredTables(); } diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/CategoryTableCondition.java b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/CategoryTableCondition.java index f3e10e6d..ae5ed243 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/CategoryTableCondition.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/CategoryTableCondition.java @@ -102,10 +102,12 @@ public Condition condition() { @Override public boolean equals(final Object object) { - if (this == object) + if (this == object) { return true; - if (object == null || getClass() != object.getClass()) + } + if (object == null || getClass() != object.getClass()) { return false; + } final CategoryTableCondition cast = (CategoryTableCondition) object; return comparedTo.equals(cast.comparedTo) && bloomTermCondition.equals(cast.bloomTermCondition) && typeIdCondition.equals(cast.typeIdCondition) && categoryTable.equals(cast.categoryTable); diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/RegexLikeCondition.java b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/RegexLikeCondition.java index 9026227b..346c0717 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/RegexLikeCondition.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/RegexLikeCondition.java @@ -89,10 +89,12 @@ public Condition condition() { @Override public boolean equals(final Object object) { - if (this == object) + if (this == object) { return true; - if (object == null || object.getClass() != this.getClass()) + } + if (object == null || object.getClass() != this.getClass()) { return false; + } final RegexLikeCondition cast = (RegexLikeCondition) object; return valueField.equals(cast.valueField) && comparedToField.equals(cast.comparedToField); } diff --git a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/ValidElement.java b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/ValidElement.java index 0ea44822..49456144 100644 --- a/src/main/java/com/teragrep/pth_06/planner/walker/conditions/ValidElement.java +++ b/src/main/java/com/teragrep/pth_06/planner/walker/conditions/ValidElement.java @@ -47,6 +47,8 @@ import org.w3c.dom.Element; +import java.util.Objects; + public final class ValidElement { private final Element element; @@ -88,16 +90,24 @@ public String operation() { @Override public boolean equals(final Object object) { - if (this == object) + if (this == object) { return true; - if (object == null) + } + if (object == null) { return false; - if (object.getClass() != this.getClass()) + } + if (object.getClass() != this.getClass()) { return false; + } final ValidElement cast = (ValidElement) object; boolean equalName = this.element.getTagName().equals(cast.element.getTagName()); boolean equalOperation = this.element.getAttribute("operation").equals(cast.element.getAttribute("operation")); boolean equalValue = this.element.getAttribute("value").equals(cast.element.getAttribute("value")); return equalName && equalOperation && equalValue; } + + @Override + public int hashCode() { + return Objects.hash(element.getTagName(), element.getAttribute("operation"), element.getAttribute("value")); + } }